diff --git a/cmd/thanos/query.go b/cmd/thanos/query.go index 13160344100..2941ede8a91 100644 --- a/cmd/thanos/query.go +++ b/cmd/thanos/query.go @@ -771,10 +771,9 @@ func runQuery( if httpProbe.IsReady() { mint, maxt := proxy.TimeRange() return &infopb.StoreInfo{ - MinTime: mint, - MaxTime: maxt, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: mint, + MaxTime: maxt, + SupportsSharding: true, } } return nil diff --git a/cmd/thanos/receive.go b/cmd/thanos/receive.go index a70edc16d3a..971151237a9 100644 --- a/cmd/thanos/receive.go +++ b/cmd/thanos/receive.go @@ -327,10 +327,10 @@ func runReceive( if httpProbe.IsReady() { minTime, maxTime := proxy.TimeRange() return &infopb.StoreInfo{ - MinTime: minTime, - MaxTime: maxTime, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: minTime, + MaxTime: maxTime, + SupportsSharding: true, + SupportsWithoutReplicaLabels: false, // TODO(bwplotka): Add support for efficiency. } } return nil diff --git a/cmd/thanos/rule.go b/cmd/thanos/rule.go index 01df5324134..4d4601d2f4e 100644 --- a/cmd/thanos/rule.go +++ b/cmd/thanos/rule.go @@ -627,10 +627,10 @@ func runRule( if httpProbe.IsReady() { mint, maxt := tsdbStore.TimeRange() return &infopb.StoreInfo{ - MinTime: mint, - MaxTime: maxt, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: mint, + MaxTime: maxt, + SupportsSharding: true, + SupportsWithoutReplicaLabels: false, // TODO(bwplotka): Add support for efficiency. } } return nil diff --git a/cmd/thanos/sidecar.go b/cmd/thanos/sidecar.go index 2206d8c9dc9..0ae99314cc8 100644 --- a/cmd/thanos/sidecar.go +++ b/cmd/thanos/sidecar.go @@ -268,10 +268,10 @@ func runSidecar( if httpProbe.IsReady() { mint, maxt := promStore.Timestamps() return &infopb.StoreInfo{ - MinTime: mint, - MaxTime: maxt, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: mint, + MaxTime: maxt, + SupportsSharding: true, + SupportsWithoutReplicaLabels: false, // TODO(bwplotka): Add support for efficiency. } } return nil diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index c59154b6b6c..46e8c52031c 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -431,10 +431,10 @@ func runStore( if httpProbe.IsReady() { mint, maxt := bs.TimeRange() return &infopb.StoreInfo{ - MinTime: mint, - MaxTime: maxt, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: mint, + MaxTime: maxt, + SupportsSharding: true, + SupportsWithoutReplicaLabels: true, } } return nil diff --git a/docs/img/bottleneck-globalsort.png b/docs/img/bottleneck-globalsort.png new file mode 100644 index 00000000000..bc1c59a28f2 Binary files /dev/null and b/docs/img/bottleneck-globalsort.png differ diff --git a/docs/img/globalsort-nonoptimized.png b/docs/img/globalsort-nonoptimized.png new file mode 100644 index 00000000000..0c81e2e45b3 Binary files /dev/null and b/docs/img/globalsort-nonoptimized.png differ diff --git a/docs/img/globalsort-optimized.png b/docs/img/globalsort-optimized.png new file mode 100644 index 00000000000..7085f9c2290 Binary files /dev/null and b/docs/img/globalsort-optimized.png differ diff --git a/docs/proposals-accepted/20221129-avoid-global-sort.md b/docs/proposals-accepted/20221129-avoid-global-sort.md new file mode 100644 index 00000000000..aff9394e34c --- /dev/null +++ b/docs/proposals-accepted/20221129-avoid-global-sort.md @@ -0,0 +1,183 @@ +## Avoid Global Sort on Querier Select + +* **Owners:** + * @bwplotka, @fpetkovski + +* **Related Tickets:** + * https://github.com/thanos-io/thanos/issues/5719 + * https://github.com/thanos-io/thanos/commit/043c5bfcc2464d3ae7af82a1428f6e0d6510f020 + * https://github.com/thanos-io/thanos/pull/5796 also alternatives (https://github.com/thanos-io/thanos/pull/5692) + +> TL;DR: We propose solution that allows saving query and query_range latency on common setups when deduplication on and data replication. Initial benchmarks indicate ~20% latency improvement for data replicated 2 times. +> +> To make it work we propose adding field to Store API Series call "WithoutReplicaLabels []string", guarded by "SupportsWithoutReplicaLabels" field propagated via Info API. It allows telling store implementations to remove given labels (if they are replica labels) from result, preserving sorting by labels after the removal. +> +> NOTE: This change will break unlikely setups that deduplicate on non-replica label (misconfiguration or wrong setup). + +## Glossary + +**replica**: We use term "replica labels" as a subset of (or equal to) "external labels": Labels that indicate unique replication group for our data, usually taken from the metadata about origin/source. + +## Why + +Currently, we spent a lof of storage selection CPU time on resorting resulting time series needed for deduplication (exactly in [`sortDedupLabels`](https://github.com/thanos-io/thanos/blob/main/pkg/query/querier.go#L400)). However, given distributed effort and current sorting guarantees of StoreAPI there is potential to reduce sorting effort or/and distribute it to leafs or multiple threads. + +### Pitfalls of the current solution + +Current flow can be represented as follows: + +![img.png](../img/bottleneck-globalsort.png) + +1. Querier PromQL Engine selects data. At this point we know if users asked for deduplicated data or not and [what replica labels to use](https://thanos.io/tip/components/query.md/#deduplication-replica-labels). +2. Querier selection asks internal, in-process Store API which is represented by Proxy code component. It asks relevant store API for data, using StoreAPI.Series. +3. Responses are pulled and k-way merged by the time series. StoreAPI guarantees the responses are sorted by series and the external labels (including replica) are included in the time series. +* There was a [bug in receiver](https://github.com/thanos-io/thanos/commit/043c5bfcc2464d3ae7af82a1428f6e0d6510f020#diff-b3f73a54121d88de203946e84955da7027e3cfce7f0cd82580bf215ac57c02f4) that caused series to be not sorted when returned. Fixed in v0.29.0. +4. Querier selection waits until all responses are buffered and then it deduplicates the data, given the requested replica labels. Before it's done it globally sort data with moving replica label at the end of the time series in `sortDedupLabels`. +5. Data is deduplicated using `dedup` package. + +The pittfall is in the fact that global sort can be in many cases completely avoided, even when deduplication is enabled. Many storeAPIs can drop certain replica labels without need to resort and others can k-way merge different data sets without certain replica labels without extra effort. + +## Goals + +Goals and use cases for the solution as proposed in [How](#how): + +* Avoid expensive global sort of all series before passing them to PromQL engine in Querier. +* Allow StoreAPI implementation to announce if it supports sorting feature or not. The rationale is that we want to make it possible to create simpler StoreAPI servers, if operator wants to trade-off it with latency. +* Clear the behaviour in tricky cases when there is an overlap of replica labels between what's in TSDB vs what's attached as external labels. +* Ensure this change can be rolled out in compatible way. + +## Non-Goals + +* Allow consuming series in streamed way in PromQL engine. + * While this pitfall (global sort) blocks the above idea, it's currently still more beneficial to pull all series upfront (eager approach) as soon as possible. This is due to current PromQL architecture which requires info upfront for query planners and execution. We don't plan to change it yet, thus no need to push explicitly for that. + +## How + +### Invariants + +To understand proposal, let's go through important, yet perhaps not trivial, facts: + +* For StoreAPI or generally data that belongs to one replica, if you exclude certain replica label during sort, it does not impact sorting order for returned series. This means, any feature that desired different sort for replicated series is generally noop for sidecars, rules, single tenant receiver or within single block (or one stream of blocks). +* You can't stream sorting of unsorted data. Furthermore, it's not possible to detect that data is unsorted, unless we fetch and buffer all series. +* In v0.29 and below, you can deduplicate on any labels, including non replicas. This is assumed semantically wrong, yet someone might depend on it. +* Thanos never handled overlap of chunks within one set of store API response. + +### Solution + +To avoid global sort, we propose removing required replica labels and sort on store API level. + +For the first step (which is required for compatibility purposes anyway), we propose a logic in proxy Store API implementation that when deduplication is requested with given replica labels will: + +* Fallback to eager retrieval. +* Remove given labels from series (this is can remove non-replica labels too, same as it is possible now). +* Resort all series (just on local level). + +Thanks of that the k-way merge will sort based on series without replica labels that will allow querier dedup to be done in streaming way without global sort and replica label removal. + +As the second step we propose adding `without_replica_labels` field to `SeriesResponse` proto message of Store API: + +```protobuf +message SeriesRequest { + // ... + + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + repeated string without_replica_labels = 14; +``` + +Since it's a new field, for compatibility we also propose adding `supports_without_replica_labels` in InfoAPI to indicate a server supports it explicitly. + +```protobuf +// StoreInfo holds the metadata related to Store API exposed by the component. +message StoreInfo { + reserved 4; // Deprecated send_sorted, replaced by supports_without_replica_labels now. + + int64 min_time = 1; + int64 max_time = 2; + bool supports_sharding = 3; + + // supports_without_replica_labels means this store supports without_replica_labels of StoreAPI.Series. + bool supports_without_replica_labels = 5; +} +``` + +Thanks of that implementations can optionally support this feature. We can make all Thanos StoreAPI support it, which will allow faster deduplication queries on all types of setups. + +In the initial tests we see 60% improvements on my test data (8M series block, requests for ~200k series) with querier and store gateway. + +Without this change: + +![1](../img/globalsort-nonoptimized.png) + +After implementing this proposal: + +![2](../img/globalsort-optimized.png) + +## Alternatives + +1. Version StoreAPI. + +As a best practice gRPC services should be versioned. This should allow easier iterations for everybody implementing or using it. However, having multiple versions (vs extra feature enablement field) might make client side more complex, so we propose to postpone it. + +2. Optimization: Add "replica group" as another message in `SeriesResponse` + +Extra slice in all Series might feel redundant, given all series are always grouped within the same replica. Let's do this once we see it being a bottleneck (will require change in StoreAPI version). + +3. Instead of removing some replica labels, just sort without them and leave at the end. + +For debugging purposes we could keep the replica labels we want to dedup on at the end of label set. + +This might however be less clean way of providing better debuggability, which is not yet required. + +Cons: +* Feels hacky. Proper way for preserving this information would be like alternative 4. +* Debuggability might be not needed here - YAGNI + +4. Replica label struct + +We could make Store API response fully replica aware. This means that series response will now include an extra slice of replica labels that this series belongs to: + +```protobuf +message Series { + repeated Label labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; + repeated Label replica_labels = 3 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; // Added. + + repeated AggrChunk chunks = 2 [(gogoproto.nullable) = false]; +} +``` + +Pros: +* Easy to tell what is replica what's not on client of Store API level + +Cons: +* Extra code and protobuf complexity +* Semantics of replica labels are hard to maintain when partial deduplication is configured (we only dedup by part of replica labels, not by all of them). This dynamic policy makes it hard to have clean response with separation of replica labels (i.e. should included replica labels be in "labels" or "replica labels")? + +This might be not needed for now. We can add more awareness of replication later on. + +## Action Plan + +The tasks to do in order to migrate to the new idea. + +* [ ] Merging the PR with the proposal (also includes implementation) +* [ ] Add support for `without_replica_label` to other store API servers. +* [ ] Move to deduplicate over chunks from series See [TODO in querier.go:405](../../pkg/query/querier.go) + +```go +// TODO(bwplotka): Move to deduplication on chunk level inside promSeriesSet, similar to what we have in dedup.NewDedupChunkMerger(). +// This however require big refactor, caring about correct AggrChunk to iterator conversion, pushdown logic and counter reset apply. +// For now we apply simple logic that splits potential overlapping chunks into separate replica series, so we can split the work. +set := &promSeriesSet{ + mint: q.mint, + maxt: q.maxt, + set: dedup.NewOverlapSplit(newStoreSeriesSet(resp.seriesSet)), + aggrs: aggrs, + warns: warns, +} +``` diff --git a/go.mod b/go.mod index da28015b291..b3fe238de2b 100644 --- a/go.mod +++ b/go.mod @@ -181,7 +181,7 @@ require ( github.com/gofrs/flock v0.8.1 // indirect github.com/gogo/googleapis v1.4.0 // indirect github.com/golang-jwt/jwt/v4 v4.4.3 // indirect - github.com/google/go-cmp v0.5.9 // indirect + github.com/google/go-cmp v0.5.9 github.com/google/go-querystring v1.1.0 // indirect github.com/google/pprof v0.0.0-20230111200839-76d1ae5aea2b // indirect github.com/google/uuid v1.3.0 // indirect diff --git a/pkg/api/query/v1_test.go b/pkg/api/query/v1_test.go index d8403522e98..edacc7c6b9c 100644 --- a/pkg/api/query/v1_test.go +++ b/pkg/api/query/v1_test.go @@ -56,6 +56,7 @@ import ( "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" + storetestutil "github.com/thanos-io/thanos/pkg/store/storepb/testutil" "github.com/thanos-io/thanos/pkg/testutil/custom" "github.com/thanos-io/thanos/pkg/testutil/e2eutil" "github.com/thanos-io/thanos/pkg/testutil/testpromcompatibility" @@ -192,7 +193,7 @@ func TestQueryEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, newProxyStoreWithTSDBStore(db), 2, timeout), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4, gate.Queries), @@ -642,6 +643,24 @@ func TestQueryEndpoints(t *testing.T) { } } +func newProxyStoreWithTSDBStore(db store.TSDBReader) *store.ProxyStore { + c := &storetestutil.TestClient{ + Name: "1", + StoreClient: storepb.ServerAsClient(store.NewTSDBStore(nil, db, component.Query, nil), 0), + MinTime: math.MinInt64, MaxTime: math.MaxInt64, + } + + return store.NewProxyStore( + nil, + nil, + func() []store.Client { return []store.Client{c} }, + component.Query, + nil, + 0, + store.EagerRetrieval, + ) +} + func TestMetadataEndpoints(t *testing.T) { var old = []labels.Labels{ { @@ -733,7 +752,7 @@ func TestMetadataEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, newProxyStoreWithTSDBStore(db), 2, timeout), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4, gate.Queries), @@ -746,7 +765,7 @@ func TestMetadataEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, newProxyStoreWithTSDBStore(db), 2, timeout), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4, gate.Queries), diff --git a/pkg/dedup/iter.go b/pkg/dedup/iter.go index 7573d00c762..49d0bf5b62c 100644 --- a/pkg/dedup/iter.go +++ b/pkg/dedup/iter.go @@ -10,12 +10,12 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/thanos-io/thanos/pkg/store/storepb" ) type dedupSeriesSet struct { - set storage.SeriesSet - replicaLabels map[string]struct{} - isCounter bool + set storage.SeriesSet + isCounter bool replicas []storage.Series // Pushed down series. Currently, they are being handled in a specific way. @@ -37,9 +37,77 @@ func isCounter(f string) bool { return f == "increase" || f == "rate" || f == "irate" || f == "resets" } -func NewSeriesSet(set storage.SeriesSet, replicaLabels map[string]struct{}, f string, pushdownEnabled bool) storage.SeriesSet { +// NewOverlapSplit splits overlapping chunks into separate series entry, so existing algorithm can work as usual. +// We cannot do this in dedup.SeriesSet as it iterates over samples already. +// TODO(bwplotka): Remove when we move to per chunk deduplication code. +// We expect non-duplicated series with sorted chunks by min time (possibly overlapped). +func NewOverlapSplit(set storepb.SeriesSet) storepb.SeriesSet { + return &overlapSplitSet{set: set, ok: true} +} + +type overlapSplitSet struct { + ok bool + set storepb.SeriesSet + + currLabels labels.Labels + currI int + replicas [][]storepb.AggrChunk +} + +func (o *overlapSplitSet) Next() bool { + if !o.ok { + return false + } + + o.currI++ + if o.currI < len(o.replicas) { + return true + } + + o.currI = 0 + o.replicas = o.replicas[:0] + o.replicas = append(o.replicas, nil) + + o.ok = o.set.Next() + if !o.ok { + return false + } + + var chunks []storepb.AggrChunk + o.currLabels, chunks = o.set.At() + if len(chunks) == 0 { + return true + } + + o.replicas[0] = append(o.replicas[0], chunks[0]) + +chunksLoop: + for i := 1; i < len(chunks); i++ { + currMinTime := chunks[i].MinTime + for ri := range o.replicas { + if len(o.replicas[ri]) == 0 || o.replicas[ri][len(o.replicas[ri])-1].MaxTime < currMinTime { + o.replicas[ri] = append(o.replicas[ri], chunks[i]) + continue chunksLoop + } + } + o.replicas = append(o.replicas, []storepb.AggrChunk{chunks[i]}) // Not found, add to a new "fake" series. + } + return true +} + +func (o *overlapSplitSet) At() (labels.Labels, []storepb.AggrChunk) { + return o.currLabels, o.replicas[o.currI] +} + +func (o *overlapSplitSet) Err() error { + return o.set.Err() +} + +// NewSeriesSet returns seriesSet that deduplicates the same series. +// The series in series set are expected be sorted by all labels. +func NewSeriesSet(set storage.SeriesSet, f string, pushdownEnabled bool) storage.SeriesSet { // TODO: remove dependency on knowing whether it is a counter. - s := &dedupSeriesSet{pushdownEnabled: pushdownEnabled, set: set, replicaLabels: replicaLabels, isCounter: isCounter(f), f: f} + s := &dedupSeriesSet{pushdownEnabled: pushdownEnabled, set: set, isCounter: isCounter(f), f: f} s.ok = s.set.Next() if s.ok { s.peek = s.set.At() @@ -63,9 +131,8 @@ func (s *dedupSeriesSet) Next() bool { } s.replicas = s.replicas[:0] - // Set the label set we are currently gathering to the peek element - // without the replica label if it exists. - s.lset = s.peekLset() + // Set the label set we are currently gathering to the peek element. + s.lset = s.peek.Labels() pushedDown := false if s.pushdownEnabled { @@ -79,28 +146,6 @@ func (s *dedupSeriesSet) Next() bool { return s.next() } -// peekLset returns the label set of the current peek element stripped from the -// replica label if it exists. -func (s *dedupSeriesSet) peekLset() labels.Labels { - lset := s.peek.Labels() - if len(s.replicaLabels) == 0 { - return lset - } - // Check how many replica labels are present so that these are removed. - var totalToRemove int - for i := 0; i < len(s.replicaLabels); i++ { - if len(lset)-i == 0 { - break - } - - if _, ok := s.replicaLabels[lset[len(lset)-i-1].Name]; ok { - totalToRemove++ - } - } - // Strip all present replica labels. - return lset[:len(lset)-totalToRemove] -} - func (s *dedupSeriesSet) next() bool { // Peek the next series to see whether it's a replica for the current series. s.ok = s.set.Next() @@ -109,7 +154,7 @@ func (s *dedupSeriesSet) next() bool { return len(s.replicas) > 0 || len(s.pushedDown) > 0 } s.peek = s.set.At() - nextLset := s.peekLset() + nextLset := s.peek.Labels() var pushedDown bool if s.pushdownEnabled { diff --git a/pkg/dedup/iter_test.go b/pkg/dedup/iter_test.go index 06da71db8b3..1dd280ec5cd 100644 --- a/pkg/dedup/iter_test.go +++ b/pkg/dedup/iter_test.go @@ -15,6 +15,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/efficientgo/core/testutil" ) @@ -145,40 +146,158 @@ var expectedRealSeriesWithStaleMarkerDeduplicatedForRate = []sample{ {t: 1587693472139, v: 509720}, {t: 1587693487139, v: 509979}, {t: 1587693502139, v: 510189}, {t: 1587693517139, v: 510505}, {t: 1587693532139, v: 510661}, {t: 1587693547139, v: 510866}, {t: 1587693562139, v: 511131}, {t: 1587693577139, v: 511321}, {t: 1587693592139, v: 511495}, } +type chunkedSeries struct { + lset labels.Labels + chunks []storepb.AggrChunk +} + +type chunkedSeriesSet struct { + series []chunkedSeries + i int +} + +func newChunkedSeriesSet(s []chunkedSeries) *chunkedSeriesSet { + return &chunkedSeriesSet{series: s, i: -1} +} + +func (s *chunkedSeriesSet) Next() bool { + if s.i >= len(s.series)-1 { + return false + } + s.i++ + return true +} + +func (*chunkedSeriesSet) Err() error { + return nil +} + +func (s *chunkedSeriesSet) At() (labels.Labels, []storepb.AggrChunk) { + return s.series[s.i].lset, s.series[s.i].chunks +} + +func toChunkedSeriesSlice(t testing.TB, set storepb.SeriesSet) []chunkedSeries { + var ret []chunkedSeries + for set.Next() { + lset, chunks := set.At() + ret = append(ret, chunkedSeries{ + lset: lset, chunks: chunks, + }) + } + testutil.Ok(t, set.Err()) + return ret +} + +func TestOverlapSplitSet(t *testing.T) { + input := []chunkedSeries{ + { + lset: labels.Labels{{Name: "a", Value: "1_empty"}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "2_nonoverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 21, MaxTime: 100}, {MinTime: 110, MaxTime: 300}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "3_tworeplicas"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 0, MaxTime: 30}, {MinTime: 21, MaxTime: 50}, {MinTime: 31, MaxTime: 60}, {MinTime: 100, MaxTime: 160}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "4_nonoverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 50, MaxTime: 55}, {MinTime: 56, MaxTime: 100}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "5_minimaloverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 50, MaxTime: 55}, {MinTime: 55, MaxTime: 100}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 0, MaxTime: 30}, {MinTime: 1, MaxTime: 15}, {MinTime: 2, MaxTime: 36}, {MinTime: 16, MaxTime: 200}, + {MinTime: 21, MaxTime: 50}, {MinTime: 31, MaxTime: 60}, {MinTime: 100, MaxTime: 160}}, + }, + } + exp := []chunkedSeries{ + { + lset: labels.Labels{{Name: "a", Value: "1_empty"}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "2_nonoverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 21, MaxTime: 100}, {MinTime: 110, MaxTime: 300}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "3_tworeplicas"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 21, MaxTime: 50}, {MinTime: 100, MaxTime: 160}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "3_tworeplicas"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 30}, {MinTime: 31, MaxTime: 60}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "4_nonoverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 50, MaxTime: 55}, {MinTime: 56, MaxTime: 100}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "5_minimaloverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 50, MaxTime: 55}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "5_minimaloverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 55, MaxTime: 100}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 21, MaxTime: 50}, {MinTime: 100, MaxTime: 160}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 30}, {MinTime: 31, MaxTime: 60}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 1, MaxTime: 15}, {MinTime: 16, MaxTime: 200}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 2, MaxTime: 36}}, + }, + } + + got := toChunkedSeriesSlice(t, NewOverlapSplit(newChunkedSeriesSet(input))) + testutil.Equals(t, exp, got) +} + func TestDedupSeriesSet(t *testing.T) { - tests := []struct { - name string - input []series - exp []series - dedupLabels map[string]struct{} - isCounter bool + for _, tcase := range []struct { + name string + input []series + exp []series + isCounter bool }{ { name: "Single dedup label", input: []series{ { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-2"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{200000, 5}, {210000, 6}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: "replica", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, }, @@ -191,10 +310,6 @@ func TestDedupSeriesSet(t *testing.T) { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, - { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, - samples: []sample{{10000, 1}, {20000, 2}}, - }, { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, @@ -204,52 +319,18 @@ func TestDedupSeriesSet(t *testing.T) { samples: []sample{{10000, 1}, {20000, 2}, {60000, 3}, {70000, 4}}, }, }, - dedupLabels: map[string]struct{}{ - "replica": {}, - }, - }, - { - // Regression tests against: https://github.com/thanos-io/thanos/issues/2645. - // We were panicking on requests with more replica labels than overall labels in any series. - name: "Regression tests against #2645", - input: []series{ - { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}}, - samples: []sample{{10000, 1}, {20000, 2}}, - }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-2"}}, - samples: []sample{{60000, 3}, {70000, 4}}, - }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}}, - samples: []sample{{100000, 10}, {150000, 20}}, - }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, - samples: []sample{{10000, 1}, {20000, 2}}, - }, - }, - exp: []series{ - { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, - samples: []sample{{10000, 1}, {20000, 2}, {60000, 3}, {70000, 4}, {100000, 10}, {150000, 20}}, - }, - { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, - samples: []sample{{10000, 1}, {20000, 2}}, - }, - }, - dedupLabels: map[string]struct{}{"replica": {}, "replica2": {}, "replica3": {}, "replica4": {}, "replica5": {}, "replica6": {}, "replica7": {}}, }, { name: "Multi dedup label", input: []series{ { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}, {Name: "replicaA", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-2"}, {Name: "replicaA", Value: "replica-2"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}, {Name: "replicaA", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{200000, 5}, {210000, 6}}, }, { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, @@ -258,13 +339,13 @@ func TestDedupSeriesSet(t *testing.T) { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: "replica", Value: "replica-1"}, {Name: "replicaA", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}, {Name: "replicaA", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}, {Name: "replicaA", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, }, @@ -290,19 +371,15 @@ func TestDedupSeriesSet(t *testing.T) { samples: []sample{{10000, 1}, {20000, 2}, {60000, 3}, {70000, 4}}, }, }, - dedupLabels: map[string]struct{}{ - "replica": {}, - "replicaA": {}, - }, }, { name: "Multi dedup label - some series don't have all dedup labels", input: []series{ { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}, {Name: "replicaA", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-2"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, }, @@ -312,10 +389,6 @@ func TestDedupSeriesSet(t *testing.T) { samples: []sample{{10000, 1}, {20000, 2}, {60000, 3}, {70000, 4}}, }, }, - dedupLabels: map[string]struct{}{ - "replica": {}, - "replicaA": {}, - }, }, { // Regression test against https://github.com/thanos-io/thanos/issues/2401. @@ -328,7 +401,7 @@ func TestDedupSeriesSet(t *testing.T) { isCounter: true, input: []series{ { - lset: labels.Labels{{Name: "replica", Value: "01"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {10000, 8.0}, // Smaller timestamp, this will be chosen. CurrValue = 8.0. {20000, 9.0}, // Same. CurrValue = 9.0. @@ -341,7 +414,7 @@ func TestDedupSeriesSet(t *testing.T) { {100000, 9 + 6.0}, }, }, { - lset: labels.Labels{{Name: "replica", Value: "02"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {10001, 8.0}, // Penalty 5000 will be added. // 20001 was app reset. No sample, because stale marker but removed by downsample.CounterSeriesIterator. Penalty 2 * (20000 - 10000) will be added. @@ -355,13 +428,10 @@ func TestDedupSeriesSet(t *testing.T) { }, exp: []series{ { - lset: labels.Labels{}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{{10000, 8}, {20000, 9}, {45001, 9}, {55001, 10}, {65001, 11}, {90000, 14}, {100000, 15}}, }, }, - dedupLabels: map[string]struct{}{ - "replica": {}, - }, }, { // Same thing but not for counter should not adjust anything. @@ -369,12 +439,12 @@ func TestDedupSeriesSet(t *testing.T) { isCounter: false, input: []series{ { - lset: labels.Labels{{Name: "replica", Value: "01"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {10000, 8.0}, {20000, 9.0}, {50001, 9 + 1.0}, {60000, 9 + 2.0}, {70000, 9 + 3.0}, {80000, 9 + 4.0}, {90000, 9 + 5.0}, {100000, 9 + 6.0}, }, }, { - lset: labels.Labels{{Name: "replica", Value: "02"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {10001, 8.0}, {45001, 8 + 0.5}, {55001, 8 + 1.5}, {65001, 8 + 2.5}, }, @@ -382,11 +452,10 @@ func TestDedupSeriesSet(t *testing.T) { }, exp: []series{ { - lset: labels.Labels{}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{{10000, 8}, {20000, 9}, {45001, 8.5}, {55001, 9.5}, {65001, 10.5}, {90000, 14}, {100000, 15}}, }, }, - dedupLabels: map[string]struct{}{"replica": {}}, }, { // Regression test on real data against https://github.com/thanos-io/thanos/issues/2401. @@ -395,7 +464,7 @@ func TestDedupSeriesSet(t *testing.T) { isCounter: true, input: []series{ { - lset: labels.Labels{{Name: "replica", Value: "01"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {t: 1587690007139, v: 461993}, {t: 1587690022139, v: 462164}, {t: 1587690037139, v: 462409}, {t: 1587690052139, v: 462662}, {t: 1587690067139, v: 462824}, {t: 1587690082139, v: 462987}, {t: 1587690097155, v: 463108}, {t: 1587690112139, v: 463261}, {t: 1587690127139, v: 463465}, {t: 1587690142139, v: 463642}, {t: 1587690157139, v: 463823}, {t: 1587690172139, v: 464065}, {t: 1587690187139, v: 464333}, {t: 1587690202139, v: 464566}, {t: 1587690217139, v: 464811}, {t: 1587690232140, v: 465032}, {t: 1587690247139, v: 465229}, {t: 1587690262139, v: 465445}, {t: 1587690277139, v: 465700}, {t: 1587690292139, v: 465884}, @@ -423,7 +492,7 @@ func TestDedupSeriesSet(t *testing.T) { {t: 1587693517139, v: 510498}, {t: 1587693532139, v: 510654}, {t: 1587693547139, v: 510859}, {t: 1587693562139, v: 511124}, {t: 1587693577139, v: 511314}, {t: 1587693592139, v: 511488}, }, }, { - lset: labels.Labels{{Name: "replica", Value: "02"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {t: 1587690005791, v: 461968}, {t: 1587690020791, v: 462151}, {t: 1587690035797, v: 462336}, {t: 1587690050791, v: 462650}, {t: 1587690065791, v: 462813}, {t: 1587690080791, v: 462987}, {t: 1587690095791, v: 463095}, {t: 1587690110791, v: 463247}, {t: 1587690125791, v: 463440}, {t: 1587690140791, v: 463642}, {t: 1587690155791, v: 463811}, {t: 1587690170791, v: 464027}, {t: 1587690185791, v: 464308}, {t: 1587690200791, v: 464514}, {t: 1587690215791, v: 464798}, {t: 1587690230791, v: 465018}, {t: 1587690245791, v: 465215}, {t: 1587690260813, v: 465431}, {t: 1587690275791, v: 465651}, {t: 1587690290791, v: 465870}, {t: 1587690305791, v: 466070}, {t: 1587690320792, v: 466248}, @@ -452,22 +521,19 @@ func TestDedupSeriesSet(t *testing.T) { }, exp: []series{ { - lset: labels.Labels{}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: expectedRealSeriesWithStaleMarkerDeduplicatedForRate, }, }, - dedupLabels: map[string]struct{}{"replica": {}}, }, - } - - for _, tcase := range tests { + } { t.Run(tcase.name, func(t *testing.T) { // If it is a counter then pass a function which expects a counter. f := "" if tcase.isCounter { f = "rate" } - dedupSet := NewSeriesSet(&mockedSeriesSet{series: tcase.input}, tcase.dedupLabels, f, false) + dedupSet := NewSeriesSet(&mockedSeriesSet{series: tcase.input}, f, false) var ats []storage.Series for dedupSet.Next() { ats = append(ats, dedupSet.At()) diff --git a/pkg/info/infopb/rpc.pb.go b/pkg/info/infopb/rpc.pb.go index e8bf57800de..6153dbb4cf5 100644 --- a/pkg/info/infopb/rpc.pb.go +++ b/pkg/info/infopb/rpc.pb.go @@ -117,10 +117,11 @@ var xxx_messageInfo_InfoResponse proto.InternalMessageInfo // StoreInfo holds the metadata related to Store API exposed by the component. type StoreInfo struct { - MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` - MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` - SupportsSharding bool `protobuf:"varint,3,opt,name=supports_sharding,json=supportsSharding,proto3" json:"supports_sharding,omitempty"` - SendsSortedSeries bool `protobuf:"varint,4,opt,name=sends_sorted_series,json=sendsSortedSeries,proto3" json:"sends_sorted_series,omitempty"` + MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` + MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` + SupportsSharding bool `protobuf:"varint,3,opt,name=supports_sharding,json=supportsSharding,proto3" json:"supports_sharding,omitempty"` + // replica_aware means this store supports without_replica_labels of StoreAPI.Series. + SupportsWithoutReplicaLabels bool `protobuf:"varint,5,opt,name=supports_without_replica_labels,json=supportsWithoutReplicaLabels,proto3" json:"supports_without_replica_labels,omitempty"` } func (m *StoreInfo) Reset() { *m = StoreInfo{} } @@ -357,40 +358,41 @@ func init() { func init() { proto.RegisterFile("info/infopb/rpc.proto", fileDescriptor_a1214ec45d2bf952) } var fileDescriptor_a1214ec45d2bf952 = []byte{ - // 519 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x93, 0xcf, 0x6a, 0xdb, 0x4e, - 0x10, 0xc7, 0xad, 0xf8, 0xff, 0x38, 0xce, 0xef, 0xe7, 0x6d, 0x5a, 0x64, 0x1f, 0x14, 0x23, 0x72, - 0x30, 0xb4, 0x48, 0xe0, 0x42, 0x29, 0xf4, 0xd4, 0x84, 0x40, 0x03, 0x0d, 0xb4, 0xb2, 0x4f, 0xb9, - 0x08, 0x39, 0x9e, 0x38, 0x02, 0x4b, 0xbb, 0xd9, 0x5d, 0x83, 0xfd, 0x16, 0x7d, 0x82, 0xbe, 0x43, - 0xdf, 0xc2, 0xc7, 0x1c, 0x7b, 0x2a, 0xad, 0xfd, 0x22, 0x45, 0xb3, 0x72, 0x6a, 0xd1, 0x9c, 0x7a, - 0x91, 0x76, 0xe7, 0xf3, 0xfd, 0x8e, 0x76, 0x67, 0x34, 0xf0, 0x3c, 0x4e, 0x6f, 0xb9, 0x9f, 0x3d, - 0xc4, 0xc4, 0x97, 0xe2, 0xc6, 0x13, 0x92, 0x6b, 0xce, 0x5a, 0xfa, 0x2e, 0x4a, 0xb9, 0xf2, 0x32, - 0xd0, 0xeb, 0x2a, 0xcd, 0x25, 0xfa, 0xf3, 0x68, 0x82, 0x73, 0x31, 0xf1, 0xf5, 0x4a, 0xa0, 0x32, - 0xba, 0xde, 0xf1, 0x8c, 0xcf, 0x38, 0x2d, 0xfd, 0x6c, 0x65, 0xa2, 0x6e, 0x1b, 0x5a, 0x97, 0xe9, - 0x2d, 0x0f, 0xf0, 0x7e, 0x81, 0x4a, 0xbb, 0xdf, 0xca, 0x70, 0x68, 0xf6, 0x4a, 0xf0, 0x54, 0x21, - 0x7b, 0x03, 0x40, 0xc9, 0x42, 0x85, 0x5a, 0xd9, 0x56, 0xbf, 0x3c, 0x68, 0x0d, 0x3b, 0x5e, 0xfe, - 0xc9, 0xeb, 0x8f, 0x19, 0x1a, 0xa1, 0x3e, 0xab, 0xac, 0x7f, 0x9c, 0x94, 0x82, 0xe6, 0x3c, 0xdf, - 0x2b, 0x76, 0x0a, 0xed, 0x73, 0x9e, 0x08, 0x9e, 0x62, 0xaa, 0xc7, 0x2b, 0x81, 0xf6, 0x41, 0xdf, - 0x1a, 0x34, 0x83, 0x62, 0x90, 0xbd, 0x82, 0x2a, 0x1d, 0xd8, 0x2e, 0xf7, 0xad, 0x41, 0x6b, 0xf8, - 0xc2, 0xdb, 0xbb, 0x8b, 0x37, 0xca, 0x08, 0x1d, 0xc6, 0x88, 0x32, 0xb5, 0x5c, 0xcc, 0x51, 0xd9, - 0x95, 0x27, 0xd4, 0x41, 0x46, 0x8c, 0x9a, 0x44, 0xec, 0x03, 0xfc, 0x97, 0xa0, 0x96, 0xf1, 0x4d, - 0x98, 0xa0, 0x8e, 0xa6, 0x91, 0x8e, 0xec, 0x2a, 0xf9, 0x4e, 0x0a, 0xbe, 0x2b, 0xd2, 0x5c, 0xe5, - 0x12, 0x4a, 0x70, 0x94, 0x14, 0x62, 0x6c, 0x08, 0x75, 0x1d, 0xc9, 0x59, 0x56, 0x80, 0x1a, 0x65, - 0xb0, 0x0b, 0x19, 0xc6, 0x86, 0x91, 0x75, 0x27, 0x64, 0x6f, 0xa1, 0x89, 0x4b, 0x4c, 0xc4, 0x3c, - 0x92, 0xca, 0xae, 0x93, 0xab, 0x57, 0x70, 0x5d, 0xec, 0x28, 0xf9, 0xfe, 0x88, 0x99, 0x0f, 0xd5, - 0xfb, 0x05, 0xca, 0x95, 0xdd, 0x20, 0x57, 0xb7, 0xe0, 0xfa, 0x9c, 0x91, 0xf7, 0x9f, 0x2e, 0xcd, - 0x45, 0x49, 0xe7, 0x7e, 0xb5, 0xa0, 0xf9, 0x58, 0x2b, 0xd6, 0x85, 0x46, 0x12, 0xa7, 0xa1, 0x8e, - 0x13, 0xb4, 0xad, 0xbe, 0x35, 0x28, 0x07, 0xf5, 0x24, 0x4e, 0xc7, 0x71, 0x82, 0x84, 0xa2, 0xa5, - 0x41, 0x07, 0x39, 0x8a, 0x96, 0x84, 0x5e, 0x42, 0x47, 0x2d, 0x84, 0xe0, 0x52, 0xab, 0x50, 0xdd, - 0x45, 0x72, 0x1a, 0xa7, 0x33, 0x6a, 0x4a, 0x23, 0xf8, 0x7f, 0x07, 0x46, 0x79, 0x9c, 0x79, 0xf0, - 0x4c, 0x61, 0x3a, 0x55, 0xa1, 0xe2, 0x52, 0xe3, 0x34, 0x54, 0x28, 0xe3, 0xbc, 0x2b, 0x8d, 0xa0, - 0x43, 0x68, 0x44, 0x64, 0x44, 0xc0, 0x6d, 0x41, 0xf3, 0xb1, 0x3b, 0xee, 0x31, 0xb0, 0xbf, 0x4b, - 0x9e, 0xfd, 0x86, 0x7b, 0x65, 0x74, 0x2f, 0xa0, 0x5d, 0xa8, 0xcf, 0xbf, 0xdd, 0xca, 0x3d, 0x82, - 0xc3, 0xfd, 0x82, 0x0d, 0xcf, 0xa1, 0x42, 0xd9, 0xde, 0xe5, 0xef, 0x62, 0x1f, 0xf7, 0xe6, 0xa0, - 0xd7, 0x7d, 0x82, 0x98, 0x89, 0x38, 0x3b, 0x5d, 0xff, 0x72, 0x4a, 0xeb, 0x8d, 0x63, 0x3d, 0x6c, - 0x1c, 0xeb, 0xe7, 0xc6, 0xb1, 0xbe, 0x6c, 0x9d, 0xd2, 0xc3, 0xd6, 0x29, 0x7d, 0xdf, 0x3a, 0xa5, - 0xeb, 0x9a, 0x99, 0xcf, 0x49, 0x8d, 0xc6, 0xeb, 0xf5, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x2e, - 0xb5, 0x07, 0x23, 0xb5, 0x03, 0x00, 0x00, + // 533 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x93, 0xdf, 0x6a, 0xdb, 0x30, + 0x14, 0xc6, 0xe3, 0xe6, 0xbf, 0xd2, 0x74, 0xad, 0xe8, 0x86, 0x13, 0x86, 0x13, 0x4c, 0x2f, 0x02, + 0x1b, 0x31, 0x64, 0x30, 0x06, 0xbb, 0x5a, 0x4b, 0x60, 0x1d, 0x2b, 0x6c, 0x4e, 0x60, 0xd0, 0x1b, + 0xa3, 0xa4, 0x6a, 0x62, 0xb0, 0x2d, 0x55, 0x92, 0x59, 0xf2, 0x16, 0x7b, 0x95, 0x5d, 0xef, 0x05, + 0x72, 0xd9, 0xcb, 0x5d, 0x8d, 0x2d, 0x79, 0x91, 0xa1, 0x23, 0x27, 0x8b, 0x59, 0xaf, 0x7a, 0x93, + 0x48, 0xe7, 0xfb, 0x7d, 0xc7, 0xd2, 0x39, 0x3a, 0xe8, 0x69, 0x98, 0xdc, 0x32, 0x4f, 0xff, 0xf0, + 0x89, 0x27, 0xf8, 0xb4, 0xcf, 0x05, 0x53, 0x0c, 0x37, 0xd4, 0x9c, 0x24, 0x4c, 0xf6, 0xb5, 0xd0, + 0x6e, 0x49, 0xc5, 0x04, 0xf5, 0x22, 0x32, 0xa1, 0x11, 0x9f, 0x78, 0x6a, 0xc9, 0xa9, 0x34, 0x5c, + 0xfb, 0x74, 0xc6, 0x66, 0x0c, 0x96, 0x9e, 0x5e, 0x99, 0xa8, 0xdb, 0x44, 0x8d, 0xcb, 0xe4, 0x96, + 0xf9, 0xf4, 0x2e, 0xa5, 0x52, 0xb9, 0xdf, 0x8b, 0xe8, 0xd0, 0xec, 0x25, 0x67, 0x89, 0xa4, 0xf8, + 0x35, 0x42, 0x90, 0x2c, 0x90, 0x54, 0x49, 0xdb, 0xea, 0x16, 0x7b, 0x8d, 0xc1, 0x49, 0x3f, 0xfb, + 0xe4, 0xf5, 0x47, 0x2d, 0x8d, 0xa8, 0x3a, 0x2f, 0xad, 0x7e, 0x75, 0x0a, 0x7e, 0x3d, 0xca, 0xf6, + 0x12, 0x9f, 0xa1, 0xe6, 0x05, 0x8b, 0x39, 0x4b, 0x68, 0xa2, 0xc6, 0x4b, 0x4e, 0xed, 0x83, 0xae, + 0xd5, 0xab, 0xfb, 0xf9, 0x20, 0x7e, 0x89, 0xca, 0x70, 0x60, 0xbb, 0xd8, 0xb5, 0x7a, 0x8d, 0xc1, + 0xb3, 0xfe, 0xde, 0x5d, 0xfa, 0x23, 0xad, 0xc0, 0x61, 0x0c, 0xa4, 0x69, 0x91, 0x46, 0x54, 0xda, + 0xa5, 0x07, 0x68, 0x5f, 0x2b, 0x86, 0x06, 0x08, 0xbf, 0x47, 0x4f, 0x62, 0xaa, 0x44, 0x38, 0x0d, + 0x62, 0xaa, 0xc8, 0x0d, 0x51, 0xc4, 0x2e, 0x83, 0xaf, 0x93, 0xf3, 0x5d, 0x01, 0x73, 0x95, 0x21, + 0x90, 0xe0, 0x28, 0xce, 0xc5, 0xf0, 0x00, 0x55, 0x15, 0x11, 0x33, 0x5d, 0x80, 0x0a, 0x64, 0xb0, + 0x73, 0x19, 0xc6, 0x46, 0x03, 0xeb, 0x16, 0xc4, 0x6f, 0x50, 0x9d, 0x2e, 0x68, 0xcc, 0x23, 0x22, + 0xa4, 0x5d, 0x05, 0x57, 0x3b, 0xe7, 0x1a, 0x6e, 0x55, 0xf0, 0xfd, 0x83, 0xb1, 0x87, 0xca, 0x77, + 0x29, 0x15, 0x4b, 0xbb, 0x06, 0xae, 0x56, 0xce, 0xf5, 0x59, 0x2b, 0xef, 0x3e, 0x5d, 0x9a, 0x8b, + 0x02, 0xe7, 0xfe, 0xb0, 0x50, 0x7d, 0x57, 0x2b, 0xdc, 0x42, 0xb5, 0x38, 0x4c, 0x02, 0x15, 0xc6, + 0xd4, 0xb6, 0xba, 0x56, 0xaf, 0xe8, 0x57, 0xe3, 0x30, 0x19, 0x87, 0x31, 0x05, 0x89, 0x2c, 0x8c, + 0x74, 0x90, 0x49, 0x64, 0x01, 0xd2, 0x0b, 0x74, 0x22, 0x53, 0xce, 0x99, 0x50, 0x32, 0x90, 0x73, + 0x22, 0x6e, 0xc2, 0x64, 0x06, 0x4d, 0xa9, 0xf9, 0xc7, 0x5b, 0x61, 0x94, 0xc5, 0xf1, 0x10, 0x75, + 0x76, 0xf0, 0xd7, 0x50, 0xcd, 0x59, 0xaa, 0x02, 0x41, 0x79, 0x14, 0x4e, 0x49, 0x00, 0x2f, 0x40, + 0x42, 0xa5, 0x6b, 0xfe, 0xf3, 0x2d, 0xf6, 0xc5, 0x50, 0xbe, 0x81, 0xe0, 0xd5, 0xc8, 0x0f, 0xa5, + 0x5a, 0xe9, 0xb8, 0xec, 0x36, 0x50, 0x7d, 0xd7, 0x3a, 0xf7, 0x14, 0xe1, 0xff, 0xfb, 0xa1, 0xdf, + 0xe8, 0x5e, 0x8d, 0xdd, 0x21, 0x6a, 0xe6, 0x8a, 0xf7, 0xb8, 0x2b, 0xbb, 0x47, 0xe8, 0x70, 0xbf, + 0x9a, 0x83, 0x0b, 0x54, 0x82, 0x6c, 0x6f, 0xb3, 0xff, 0x7c, 0x93, 0xf7, 0x86, 0xa4, 0xdd, 0x7a, + 0x40, 0x31, 0xe3, 0x72, 0x7e, 0xb6, 0xfa, 0xe3, 0x14, 0x56, 0x6b, 0xc7, 0xba, 0x5f, 0x3b, 0xd6, + 0xef, 0xb5, 0x63, 0x7d, 0xdb, 0x38, 0x85, 0xfb, 0x8d, 0x53, 0xf8, 0xb9, 0x71, 0x0a, 0xd7, 0x15, + 0x33, 0xbc, 0x93, 0x0a, 0xcc, 0xde, 0xab, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x7f, 0x1d, 0x6e, + 0xa7, 0xd2, 0x03, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -634,15 +636,15 @@ func (m *StoreInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.SendsSortedSeries { + if m.SupportsWithoutReplicaLabels { i-- - if m.SendsSortedSeries { + if m.SupportsWithoutReplicaLabels { dAtA[i] = 1 } else { dAtA[i] = 0 } i-- - dAtA[i] = 0x20 + dAtA[i] = 0x28 } if m.SupportsSharding { i-- @@ -870,7 +872,7 @@ func (m *StoreInfo) Size() (n int) { if m.SupportsSharding { n += 2 } - if m.SendsSortedSeries { + if m.SupportsWithoutReplicaLabels { n += 2 } return n @@ -1402,9 +1404,9 @@ func (m *StoreInfo) Unmarshal(dAtA []byte) error { } } m.SupportsSharding = bool(v != 0) - case 4: + case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SendsSortedSeries", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SupportsWithoutReplicaLabels", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -1421,7 +1423,7 @@ func (m *StoreInfo) Unmarshal(dAtA []byte) error { break } } - m.SendsSortedSeries = bool(v != 0) + m.SupportsWithoutReplicaLabels = bool(v != 0) default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) diff --git a/pkg/info/infopb/rpc.proto b/pkg/info/infopb/rpc.proto index 6ce9caecc23..90cecd58905 100644 --- a/pkg/info/infopb/rpc.proto +++ b/pkg/info/infopb/rpc.proto @@ -53,10 +53,14 @@ message InfoResponse { // StoreInfo holds the metadata related to Store API exposed by the component. message StoreInfo { + reserved 4; // Deprecated send_sorted, replaced by supports_without_replica_labels now. + int64 min_time = 1; int64 max_time = 2; - bool supports_sharding = 3; - bool sends_sorted_series = 4; + bool supports_sharding = 3; + + // replica_aware means this store supports without_replica_labels of StoreAPI.Series. + bool supports_without_replica_labels = 5; } // RulesInfo holds the metadata related to Rules API exposed by the component. diff --git a/pkg/query/endpointset.go b/pkg/query/endpointset.go index 0a2d21cf2bf..e58f891cc0f 100644 --- a/pkg/query/endpointset.go +++ b/pkg/query/endpointset.go @@ -190,7 +190,7 @@ type EndpointStatus struct { // endpointSetNodeCollector is a metric collector reporting the number of available storeAPIs for Querier. // A Collector is required as we want atomic updates for all 'thanos_store_nodes_grpc_connections' series. -// TODO(hitanshu-mehta) Currently,only collecting metrics of storeAPI. Make this struct generic. +// TODO(hitanshu-mehta) Currently,only collecting metrics of storeEndpoints. Make this struct generic. type endpointSetNodeCollector struct { mtx sync.Mutex storeNodes map[component.Component]map[string]int @@ -436,7 +436,7 @@ func (e *EndpointSet) Update(ctx context.Context) { if er.HasStoreAPI() && (er.ComponentType() == component.Sidecar || er.ComponentType() == component.Rule) && stats[component.Sidecar][extLset]+stats[component.Rule][extLset] > 0 { - level.Warn(e.logger).Log("msg", "found duplicate storeAPI producer (sidecar or ruler). This is not advices as it will malform data in in the same bucket", + level.Warn(e.logger).Log("msg", "found duplicate storeEndpoints producer (sidecar or ruler). This is not advices as it will malform data in in the same bucket", "address", addr, "extLset", extLset, "duplicates", fmt.Sprintf("%v", stats[component.Sidecar][extLset]+stats[component.Rule][extLset]+1)) } stats[er.ComponentType()][extLset]++ @@ -807,7 +807,7 @@ func (er *endpointRef) SupportsSharding() bool { return er.metadata.Store.SupportsSharding } -func (er *endpointRef) SendsSortedSeries() bool { +func (er *endpointRef) SupportsWithoutReplicaLabels() bool { er.mtx.RLock() defer er.mtx.RUnlock() @@ -815,13 +815,13 @@ func (er *endpointRef) SendsSortedSeries() bool { return false } - return er.metadata.Store.SendsSortedSeries + return er.metadata.Store.SupportsWithoutReplicaLabels } func (er *endpointRef) String() string { mint, maxt := er.TimeRange() return fmt.Sprintf( - "Addr: %s LabelSets: %v Mint: %d Maxt: %d", + "Addr: %s LabelSets: %v MinTime: %d MaxTime: %d", er.addr, labelpb.PromLabelSetsToString(er.LabelSets()), mint, maxt, ) } @@ -838,7 +838,7 @@ func (er *endpointRef) apisPresent() []string { var apisPresent []string if er.HasStoreAPI() { - apisPresent = append(apisPresent, "storeAPI") + apisPresent = append(apisPresent, "storeEndpoints") } if er.HasRulesAPI() { diff --git a/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go b/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go index 112d855eaea..0e78a4ccd9e 100644 --- a/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go +++ b/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go @@ -208,14 +208,14 @@ func (s *storeRef) SupportsSharding() bool { return false } -func (s *storeRef) SendsSortedSeries() bool { +func (s *storeRef) SupportsWithoutReplicaLabels() bool { return false } func (s *storeRef) String() string { mint, maxt := s.TimeRange() return fmt.Sprintf( - "Addr: %s LabelSets: %v Mint: %d Maxt: %d", + "Addr: %s LabelSets: %v MinTime: %d MaxTime: %d", s.addr, labelpb.PromLabelSetsToString(s.LabelSets()), mint, maxt, ) } diff --git a/pkg/query/iter.go b/pkg/query/iter.go index 9b6b79e9a84..2a4e90b92b7 100644 --- a/pkg/query/iter.go +++ b/pkg/query/iter.go @@ -4,8 +4,6 @@ package query import ( - "sort" - "github.com/pkg/errors" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" @@ -18,87 +16,27 @@ import ( ) // promSeriesSet implements the SeriesSet interface of the Prometheus storage -// package on top of our storepb SeriesSet. +// package on top of our storepb SeriesSet. Overlapping chunks will be naively deduplicated (random selection). type promSeriesSet struct { - set storepb.SeriesSet - done bool + set storepb.SeriesSet mint, maxt int64 aggrs []storepb.Aggr - initiated bool - - currLset labels.Labels - currChunks []storepb.AggrChunk warns storage.Warnings } func (s *promSeriesSet) Next() bool { - if !s.initiated { - s.initiated = true - s.done = s.set.Next() - } - - if !s.done { - return false - } - - // storage.Series are more strict then SeriesSet: - // * It requires storage.Series to iterate over full series. - s.currLset, s.currChunks = s.set.At() - for { - s.done = s.set.Next() - if !s.done { - break - } - nextLset, nextChunks := s.set.At() - if labels.Compare(s.currLset, nextLset) != 0 { - break - } - s.currChunks = append(s.currChunks, nextChunks...) - } - - // Samples (so chunks as well) have to be sorted by time. - // TODO(bwplotka): Benchmark if we can do better. - // For example we could iterate in above loop and write our own binary search based insert sort. - // We could also remove duplicates in same loop. - sort.Slice(s.currChunks, func(i, j int) bool { - return s.currChunks[i].MinTime < s.currChunks[j].MinTime - }) - - // Proxy handles duplicates between different series, let's handle duplicates within single series now as well. - // We don't need to decode those. - s.currChunks = removeExactDuplicates(s.currChunks) - return true -} - -// removeExactDuplicates returns chunks without 1:1 duplicates. -// NOTE: input chunks has to be sorted by minTime. -func removeExactDuplicates(chks []storepb.AggrChunk) []storepb.AggrChunk { - if len(chks) <= 1 { - return chks - } - head := 0 - for i, c := range chks[1:] { - if chks[head].Compare(c) == 0 { - continue - } - head++ - if i+1 == head { - // `chks[head] == chks[i+1] == c` so this is a no-op. - // This way we get no copies in case the input had no duplicates. - continue - } - chks[head] = c - } - return chks[:head+1] + return s.set.Next() } func (s *promSeriesSet) At() storage.Series { - if !s.initiated || s.set.Err() != nil { + if s.set.Err() != nil { return nil } - return newChunkSeries(s.currLset, s.currChunks, s.mint, s.maxt, s.aggrs) + + currLset, currChunks := s.set.At() + return newChunkSeries(currLset, currChunks, s.mint, s.maxt, s.aggrs) } func (s *promSeriesSet) Err() error { @@ -128,11 +66,11 @@ func (s *storeSeriesSet) Next() bool { return true } -func (storeSeriesSet) Err() error { +func (*storeSeriesSet) Err() error { return nil } -func (s storeSeriesSet) At() (labels.Labels, []storepb.AggrChunk) { +func (s *storeSeriesSet) At() (labels.Labels, []storepb.AggrChunk) { return s.series[s.i].PromLabels(), s.series[s.i].Chunks } diff --git a/pkg/query/iter_test.go b/pkg/query/iter_test.go deleted file mode 100644 index 5282f102a18..00000000000 --- a/pkg/query/iter_test.go +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package query - -import ( - "fmt" - "testing" - - "github.com/efficientgo/core/testutil" - "github.com/thanos-io/thanos/pkg/store/storepb" -) - -func TestRemoveExactDuplicates(t *testing.T) { - for _, tc := range []struct { - name string - chks []storepb.AggrChunk - }{ - {name: "empty slice", chks: []storepb.AggrChunk{}}, - {name: "single element slice", chks: aggrChunkForMinTimes(0)}, - {name: "slice with duplicates", chks: aggrChunkForMinTimes(0, 0, 2, 30, 31, 31, 40)}, - {name: "slice without duplicates", chks: aggrChunkForMinTimes(0, 1, 2, 3, 5, 7, 8)}, - } { - t.Run(tc.name, func(t *testing.T) { - originalChks := make([]storepb.AggrChunk, len(tc.chks)) - copy(originalChks, tc.chks) - chks := removeExactDuplicates(tc.chks) - missingChunk := isChunksSubset(originalChks, chks) - testutil.Assert(t, missingChunk == nil, fmt.Sprintf("chunk %q missing in output", missingChunk.String())) - unexpectedChunk := isChunksSubset(chks, originalChks) - testutil.Assert(t, unexpectedChunk == nil, fmt.Sprintf("unexpected chunk %q does not appear in the input", unexpectedChunk.String())) - - if len(chks) > 0 { - chk1 := chks[0] - for _, chk2 := range chks[1:] { - testutil.Assert(t, chk2.Compare(chk1) != 0, fmt.Sprintf("chunk %q appears twice in output", chk1.String())) - chk1 = chk2 - } - } - }) - } -} - -func aggrChunkForMinTimes(minTimes ...int64) []storepb.AggrChunk { - chks := make([]storepb.AggrChunk, len(minTimes)) - for i, minTime := range minTimes { - chks[i] = storepb.AggrChunk{MinTime: minTime} - } - return chks -} - -// isChunksSubset returns nil if all chunks in chks1 also appear in chks2, -// otherwise returns a chunk in chks1 that does not apper in chks2. -func isChunksSubset(chks1, chks2 []storepb.AggrChunk) *storepb.AggrChunk { - for _, chk1 := range chks1 { - found := false - for _, chk2 := range chks2 { - if chk2.Compare(chk1) == 0 { - found = true - break - } - } - if !found { - return &chk1 - } - } - return nil -} diff --git a/pkg/query/querier.go b/pkg/query/querier.go index e3b75704378..074a9f0d3d5 100644 --- a/pkg/query/querier.go +++ b/pkg/query/querier.go @@ -5,7 +5,6 @@ package query import ( "context" - "sort" "strings" "sync" "time" @@ -17,12 +16,10 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" - "github.com/thanos-io/thanos/pkg/dedup" "github.com/thanos-io/thanos/pkg/extprom" "github.com/thanos-io/thanos/pkg/gate" "github.com/thanos-io/thanos/pkg/store" - "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/tracing" ) @@ -59,10 +56,11 @@ type QueryableCreator func( ) storage.Queryable // NewQueryableCreator creates QueryableCreator. +// NOTE(bwplotka): Proxy assumes to be replica_aware, see thanos.store.info.StoreInfo.replica_aware field. func NewQueryableCreator( logger log.Logger, reg prometheus.Registerer, - proxy storepb.StoreServer, + proxy *store.ProxyStore, maxConcurrentSelects int, selectTimeout time.Duration, ) QueryableCreator { @@ -104,7 +102,7 @@ type queryable struct { logger log.Logger replicaLabels []string storeDebugMatchers [][]*labels.Matcher - proxy storepb.StoreServer + proxy *store.ProxyStore deduplicate bool maxResolutionMillis int64 partialResponse bool @@ -123,22 +121,22 @@ func (q *queryable) Querier(ctx context.Context, mint, maxt int64) (storage.Quer } type querier struct { - ctx context.Context - logger log.Logger - cancel func() - mint, maxt int64 - replicaLabels map[string]struct{} - storeDebugMatchers [][]*labels.Matcher - proxy storepb.StoreServer - deduplicate bool - maxResolutionMillis int64 - partialResponse bool - enableQueryPushdown bool - skipChunks bool - selectGate gate.Gate - selectTimeout time.Duration - shardInfo *storepb.ShardInfo - seriesStatsReporter seriesStatsReporter + ctx context.Context + logger log.Logger + cancel func() + mint, maxt int64 + replicaLabels []string + storeDebugMatchers [][]*labels.Matcher + proxy *store.ProxyStore + deduplicate bool + maxResolutionMillis int64 + partialResponseStrategy storepb.PartialResponseStrategy + enableQueryPushdown bool + skipChunks bool + selectGate gate.Gate + selectTimeout time.Duration + shardInfo *storepb.ShardInfo + seriesStatsReporter seriesStatsReporter } // newQuerier creates implementation of storage.Querier that fetches data from the proxy @@ -150,7 +148,7 @@ func newQuerier( maxt int64, replicaLabels []string, storeDebugMatchers [][]*labels.Matcher, - proxy storepb.StoreServer, + proxy *store.ProxyStore, deduplicate bool, maxResolutionMillis int64, partialResponse, @@ -170,6 +168,11 @@ func newQuerier( for _, replicaLabel := range replicaLabels { rl[replicaLabel] = struct{}{} } + + partialResponseStrategy := storepb.PartialResponseStrategy_ABORT + if partialResponse { + partialResponseStrategy = storepb.PartialResponseStrategy_WARN + } return &querier{ ctx: ctx, logger: logger, @@ -177,18 +180,18 @@ func newQuerier( selectGate: selectGate, selectTimeout: selectTimeout, - mint: mint, - maxt: maxt, - replicaLabels: rl, - storeDebugMatchers: storeDebugMatchers, - proxy: proxy, - deduplicate: deduplicate, - maxResolutionMillis: maxResolutionMillis, - partialResponse: partialResponse, - skipChunks: skipChunks, - enableQueryPushdown: enableQueryPushdown, - shardInfo: shardInfo, - seriesStatsReporter: seriesStatsReporter, + mint: mint, + maxt: maxt, + replicaLabels: replicaLabels, + storeDebugMatchers: storeDebugMatchers, + proxy: proxy, + deduplicate: deduplicate, + maxResolutionMillis: maxResolutionMillis, + partialResponseStrategy: partialResponseStrategy, + skipChunks: skipChunks, + enableQueryPushdown: enableQueryPushdown, + shardInfo: shardInfo, + seriesStatsReporter: seriesStatsReporter, } } @@ -276,8 +279,9 @@ func (q *querier) Select(_ bool, hints *storage.SelectHints, ms ...*labels.Match matchers[i] = m.String() } - // The querier has a context but it gets canceled, as soon as query evaluation is completed, by the engine. + // The querier has a context, but it gets canceled as soon as query evaluation is completed by the engine. // We want to prevent this from happening for the async store API calls we make while preserving tracing context. + // TODO(bwplotka): Does the above still is true? It feels weird to leave unfinished calls behind query API. ctx := tracing.CopyTraceContext(context.Background(), q.ctx) ctx, cancel := context.WithTimeout(ctx, q.selectTimeout) span, ctx := tracing.StartSpan(ctx, "querier_select", opentracing.Tags{ @@ -337,26 +341,29 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . // TODO(bwplotka): Pass it using the SeriesRequest instead of relying on context. ctx = context.WithValue(ctx, store.StoreMatcherKey, q.storeDebugMatchers) - // TODO(bwplotka): Use inprocess gRPC. + // TODO(bwplotka): Use inprocess gRPC when we want to stream responses. + // Currently streaming won't help due to nature of the both PromQL engine which + // pulls all series before computations anyway. resp := &seriesServer{ctx: ctx} - var queryHints *storepb.QueryHints - if q.enableQueryPushdown { - queryHints = storeHintsFromPromHints(hints) - } - - if err := q.proxy.Series(&storepb.SeriesRequest{ - MinTime: q.mint, - MaxTime: q.maxt, + req := storepb.SeriesRequest{ + MinTime: hints.Start, + MaxTime: hints.End, Matchers: sms, MaxResolutionWindow: q.maxResolutionMillis, Aggregates: aggrs, - QueryHints: queryHints, ShardInfo: q.shardInfo, - PartialResponseDisabled: !q.partialResponse, + PartialResponseStrategy: q.partialResponseStrategy, SkipChunks: q.skipChunks, - Step: hints.Step, - Range: hints.Range, - }, resp); err != nil { + } + if q.enableQueryPushdown { + req.QueryHints = storeHintsFromPromHints(hints) + } + if q.isDedupEnabled() { + // Soft ask to sort without replica labels and push them at the end of labelset. + req.WithoutReplicaLabels = q.replicaLabels + } + + if err := q.proxy.Series(&req, resp); err != nil { return nil, storepb.SeriesStatsCounter{}, errors.Wrap(err, "proxy Series()") } @@ -365,10 +372,10 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . warns = append(warns, errors.New(w)) } - // Delete the metric's name from the result because that's what the - // PromQL does either way and we want our iterator to work with data - // that was either pushed down or not. if q.enableQueryPushdown && (hints.Func == "max_over_time" || hints.Func == "min_over_time") { + // On query pushdown, delete the metric's name from the result because that's what the + // PromQL does either way, and we want our iterator to work with data + // that was either pushed down or not. for i := range resp.seriesSet { lbls := resp.seriesSet[i].Labels for j, lbl := range lbls { @@ -382,7 +389,6 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . } if !q.isDedupEnabled() { - // Return data without any deduplication. return &promSeriesSet{ mint: q.mint, maxt: q.maxt, @@ -392,48 +398,18 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . }, resp.seriesSetStats, nil } - // TODO(fabxc): this could potentially pushed further down into the store API to make true streaming possible. - sortDedupLabels(resp.seriesSet, q.replicaLabels) + // TODO(bwplotka): Move to deduplication on chunk level inside promSeriesSet, similar to what we have in dedup.NewDedupChunkMerger(). + // This however require big refactor, caring about correct AggrChunk to iterator conversion, pushdown logic and counter reset apply. + // For now we apply simple logic that splits potential overlapping chunks into separate replica series, so we can split the work. set := &promSeriesSet{ mint: q.mint, maxt: q.maxt, - set: newStoreSeriesSet(resp.seriesSet), + set: dedup.NewOverlapSplit(newStoreSeriesSet(resp.seriesSet)), aggrs: aggrs, warns: warns, } - // The merged series set assembles all potentially-overlapping time ranges of the same series into a single one. - // TODO(bwplotka): We could potentially dedup on chunk level, use chunk iterator for that when available. - return dedup.NewSeriesSet(set, q.replicaLabels, hints.Func, q.enableQueryPushdown), resp.seriesSetStats, nil -} - -// sortDedupLabels re-sorts the set so that the same series with different replica -// labels are coming right after each other. -func sortDedupLabels(set []storepb.Series, replicaLabels map[string]struct{}) { - for _, s := range set { - // Move the replica labels to the very end. - sort.Slice(s.Labels, func(i, j int) bool { - if _, ok := replicaLabels[s.Labels[i].Name]; ok { - return false - } - if _, ok := replicaLabels[s.Labels[j].Name]; ok { - return true - } - // Ensure that dedup marker goes just right before the replica labels. - if s.Labels[i].Name == dedup.PushdownMarker.Name { - return false - } - if s.Labels[j].Name == dedup.PushdownMarker.Name { - return true - } - return s.Labels[i].Name < s.Labels[j].Name - }) - } - // With the re-ordered label sets, re-sorting all series aligns the same series - // from different replicas sequentially. - sort.Slice(set, func(i, j int) bool { - return labels.Compare(labelpb.ZLabelsToPromLabels(set[i].Labels), labelpb.ZLabelsToPromLabels(set[j].Labels)) < 0 - }) + return dedup.NewSeriesSet(set, hints.Func, q.enableQueryPushdown), resp.seriesSetStats, nil } // LabelValues returns all potential values for a label name. @@ -451,7 +427,7 @@ func (q *querier) LabelValues(name string, matchers ...*labels.Matcher) ([]strin resp, err := q.proxy.LabelValues(ctx, &storepb.LabelValuesRequest{ Label: name, - PartialResponseDisabled: !q.partialResponse, + PartialResponseStrategy: q.partialResponseStrategy, Start: q.mint, End: q.maxt, Matchers: pbMatchers, @@ -483,7 +459,7 @@ func (q *querier) LabelNames(matchers ...*labels.Matcher) ([]string, storage.War } resp, err := q.proxy.LabelNames(ctx, &storepb.LabelNamesRequest{ - PartialResponseDisabled: !q.partialResponse, + PartialResponseStrategy: q.partialResponseStrategy, Start: q.mint, End: q.maxt, Matchers: pbMatchers, diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 7affdf34083..65cf841381b 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -12,10 +12,13 @@ import ( "reflect" "sort" "strconv" + "strings" "testing" "time" + "github.com/efficientgo/core/testutil" "github.com/go-kit/log" + "github.com/google/go-cmp/cmp" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/histogram" @@ -26,14 +29,11 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/gate" - - "github.com/efficientgo/core/testutil" - "github.com/thanos-io/thanos/pkg/component" - "github.com/thanos-io/thanos/pkg/dedup" "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" + storetestutil "github.com/thanos-io/thanos/pkg/store/storepb/testutil" ) type sample struct { @@ -43,7 +43,7 @@ type sample struct { func TestQueryableCreator_MaxResolution(t *testing.T) { testProxy := &testStoreServer{resps: []*storepb.SeriesResponse{}} - queryableCreator := NewQueryableCreator(nil, nil, testProxy, 2, 5*time.Second) + queryableCreator := NewQueryableCreator(nil, nil, newProxyStore(testProxy), 2, 5*time.Second) oneHourMillis := int64(1*time.Hour) / int64(time.Millisecond) queryable := queryableCreator( @@ -86,7 +86,7 @@ func TestQuerier_DownsampledData(t *testing.T) { q := NewQueryableCreator( nil, nil, - testProxy, + newProxyStore(testProxy), 2, timeout, )(false, @@ -338,7 +338,7 @@ func TestQuerier_Select_AfterPromQL(t *testing.T) { for _, tcase := range []struct { name string - storeAPI storepb.StoreServer + storeAPI *store.ProxyStore replicaLabels []string // Replica label groups chunks by the label value and strips it from the final result. hints *storage.SelectHints equivalentQuery string @@ -350,11 +350,11 @@ func TestQuerier_Select_AfterPromQL(t *testing.T) { { // Regression test 1 against https://github.com/thanos-io/thanos/issues/2890. name: "when switching replicas don't miss samples when set with a big enough lookback delta", - storeAPI: func() storepb.StoreServer { + storeAPI: newProxyStore(func() storepb.StoreServer { s, err := store.NewLocalStoreFromJSONMmappableFile(logger, component.Debug, nil, "./testdata/issue2890-seriesresponses.json", store.ScanGRPCCurlProtoStreamMessages) testutil.Ok(t, err) return s - }(), + }()), equivalentQuery: `cluster_version{}`, replicaLabels: []string{"replica"}, hints: &storage.SelectHints{ @@ -425,8 +425,8 @@ func TestQuerier_Select(t *testing.T) { logger := log.NewLogfmtLogger(os.Stderr) for _, tcase := range []struct { - name string - storeAPI storepb.StoreServer + name string + storeEndpoints []storepb.StoreServer mint, maxt int64 matchers []*labels.Matcher @@ -435,29 +435,35 @@ func TestQuerier_Select(t *testing.T) { equivalentQuery string expected []series - expectedAfterDedup series + expectedAfterDedup []series expectedWarning string }{ { name: "select overlapping data with partial error", - storeAPI: &testStoreServer{ - resps: []*storepb.SeriesResponse{ - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), - storepb.NewWarnSeriesResponse(errors.New("partial error")), - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 6}, {7, 7}}), - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 66}}), // Overlap samples for some reason. - storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), - storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{100, 1}, {300, 3}, {400, 4}}), + storeEndpoints: []storepb.StoreServer{ + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storepb.NewWarnSeriesResponse(errors.New("partial error")), + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 66}}), + storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{100, 1}, {300, 3}, {400, 4}}), + }, }, }, mint: 1, maxt: 300, replicaLabels: []string{"a"}, equivalentQuery: `{a=~"a|b|c"}`, - + matchers: []*labels.Matcher{{ + Value: "a|b|c", + Name: "a", + Type: labels.MatchRegexp, + }}, expected: []series{ { lset: labels.FromStrings("a", "a"), - samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 66}, {7, 7}}, }, { lset: labels.FromStrings("a", "b"), @@ -468,20 +474,20 @@ func TestQuerier_Select(t *testing.T) { samples: []sample{{100, 1}, {300, 3}}, }, }, - expectedAfterDedup: series{ - lset: labels.Labels{}, + expectedAfterDedup: []series{{ + lset: nil, // We don't expect correctness here, it's just random non-replica data. - samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}}, - }, + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {5, 5}, {6, 6}, {7, 7}}, + }}, expectedWarning: "partial error", }, { name: "realistic data with stale marker", - storeAPI: func() storepb.StoreServer { + storeEndpoints: []storepb.StoreServer{func() storepb.StoreServer { s, err := store.NewLocalStoreFromJSONMmappableFile(logger, component.Debug, nil, "./testdata/issue2401-seriesresponses.json", store.ScanGRPCCurlProtoStreamMessages) testutil.Ok(t, err) return s - }(), + }()}, mint: realSeriesWithStaleMarkerMint, maxt: realSeriesWithStaleMarkerMaxt, replicaLabels: []string{"replica"}, matchers: []*labels.Matcher{{ @@ -509,7 +515,7 @@ func TestQuerier_Select(t *testing.T) { samples: expectedRealSeriesWithStaleMarkerReplica1, }, }, - expectedAfterDedup: series{ + expectedAfterDedup: []series{{ lset: labels.FromStrings( // No replica label anymore. "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", @@ -517,15 +523,15 @@ func TestQuerier_Select(t *testing.T) { "gcp", "region", "us-east", "shard", "default", "stage", "main", "tier", "sv", "type", "web", ), samples: expectedRealSeriesWithStaleMarkerDeduplicated, - }, + }}, }, { name: "realistic data with stale marker with 100000 step", - storeAPI: func() storepb.StoreServer { + storeEndpoints: []storepb.StoreServer{func() storepb.StoreServer { s, err := store.NewLocalStoreFromJSONMmappableFile(logger, component.Debug, nil, "./testdata/issue2401-seriesresponses.json", store.ScanGRPCCurlProtoStreamMessages) testutil.Ok(t, err) return s - }(), + }()}, mint: realSeriesWithStaleMarkerMint, maxt: realSeriesWithStaleMarkerMaxt, replicaLabels: []string{"replica"}, matchers: []*labels.Matcher{{ @@ -558,7 +564,7 @@ func TestQuerier_Select(t *testing.T) { samples: expectedRealSeriesWithStaleMarkerReplica1, }, }, - expectedAfterDedup: series{ + expectedAfterDedup: []series{{ lset: labels.FromStrings( // No replica label anymore. "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", @@ -566,17 +572,17 @@ func TestQuerier_Select(t *testing.T) { "gcp", "region", "us-east", "shard", "default", "stage", "main", "tier", "sv", "type", "web", ), samples: expectedRealSeriesWithStaleMarkerDeduplicated, - }, + }}, }, { // Regression test against https://github.com/thanos-io/thanos/issues/2401. // Thanks to @Superq and GitLab for real data reproducing this. name: "realistic data with stale marker with hints rate function", - storeAPI: func() storepb.StoreServer { + storeEndpoints: []storepb.StoreServer{func() storepb.StoreServer { s, err := store.NewLocalStoreFromJSONMmappableFile(logger, component.Debug, nil, "./testdata/issue2401-seriesresponses.json", store.ScanGRPCCurlProtoStreamMessages) testutil.Ok(t, err) return s - }(), + }()}, mint: realSeriesWithStaleMarkerMint, maxt: realSeriesWithStaleMarkerMaxt, replicaLabels: []string{"replica"}, matchers: []*labels.Matcher{{ @@ -610,13 +616,141 @@ func TestQuerier_Select(t *testing.T) { samples: expectedRealSeriesWithStaleMarkerReplica1ForRate, }, }, - expectedAfterDedup: series{ + expectedAfterDedup: []series{{ lset: labels.FromStrings( "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", "gprd", "fqdn", "web-08-sv-gprd.c.gitlab-production.internal", "instance", "web-08-sv-gprd.c.gitlab-production.internal:8083", "job", "gitlab-rails", "monitor", "app", "provider", "gcp", "region", "us-east", "shard", "default", "stage", "main", "tier", "sv", "type", "web", ), samples: expectedRealSeriesWithStaleMarkerDeduplicatedForRate, + }}, + }, + // Tests with proxy (integration test with store.ProxyStore). + { + name: "select with proxied Store APIs that does not support without replica label", + storeEndpoints: []storepb.StoreServer{ + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "w", "1"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "x", "1"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "x", "1"), []sample{{100, 1}, {300, 3}, {400, 4}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{5, 5}, {7, 7}}), + }, + }, + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{2, 1}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "x", "2"), []sample{{10, 10}, {30, 30}, {40, 40}}), + }, + }, + }, + mint: 1, maxt: 300, + replicaLabels: []string{"r"}, + equivalentQuery: `{a=~"1"}`, + matchers: []*labels.Matcher{{Name: "a", Value: "1", Type: labels.MatchRegexp}}, + expected: []series{ + { + lset: labels.FromStrings("a", "1", "r", "1", "w", "1"), + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "1", "x", "1"), + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {100, 1}, {300, 3}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "2", "w", "1"), + samples: []sample{{2, 1}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "2", "x", "2"), + samples: []sample{{10, 10}, {30, 30}, {40, 40}}, + }, + }, + expectedAfterDedup: []series{ + { + lset: labels.FromStrings("a", "1", "w", "1"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "1"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {100, 1}, {300, 3}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "2"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{10, 10}, {30, 30}, {40, 40}}, + }, + }, + }, + { + name: "select with proxied Store APIs with some stores supporting without replica labels feature", + storeEndpoints: []storepb.StoreServer{ + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "w", "1"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "x", "1"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "x", "1"), []sample{{100, 1}, {300, 3}, {400, 4}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{5, 5}, {7, 7}}), + }, + respsWithoutReplicaLabels: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "1"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "1"), []sample{{100, 1}, {300, 3}, {400, 4}}), + }, + }, + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{2, 1}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "x", "2"), []sample{{10, 10}, {30, 30}, {40, 40}}), + }, + }, + }, + mint: 1, maxt: 300, + replicaLabels: []string{"r"}, + equivalentQuery: `{a=~"1"}`, + matchers: []*labels.Matcher{{Name: "a", Value: "1", Type: labels.MatchRegexp}}, + expected: []series{ + { + lset: labels.FromStrings("a", "1", "r", "1", "w", "1"), + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "1", "x", "1"), + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {100, 1}, {300, 3}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "2", "w", "1"), + samples: []sample{{2, 1}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "2", "x", "2"), + samples: []sample{{10, 10}, {30, 30}, {40, 40}}, + }, + }, + expectedAfterDedup: []series{ + { + lset: labels.FromStrings("a", "1", "w", "1"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "1"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {100, 1}, {300, 3}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "2"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{10, 10}, {30, 30}, {40, 40}}, + }, }, }, } { @@ -633,10 +767,27 @@ func TestQuerier_Select(t *testing.T) { expected []series }{ {dedup: false, expected: tcase.expected}, - {dedup: true, expected: []series{tcase.expectedAfterDedup}}, + {dedup: true, expected: tcase.expectedAfterDedup}, } { g := gate.New(2) - q := newQuerier(context.Background(), nil, tcase.mint, tcase.maxt, tcase.replicaLabels, nil, tcase.storeAPI, sc.dedup, 0, true, false, false, g, timeout, nil, func(i storepb.SeriesStatsCounter) {}) + q := newQuerier( + context.Background(), + nil, + tcase.mint, + tcase.maxt, + tcase.replicaLabels, + nil, + newProxyStore(tcase.storeEndpoints...), + sc.dedup, + 0, + true, + false, + false, + g, + timeout, + nil, + NoopSeriesStatsReporter, + ) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) t.Run(fmt.Sprintf("dedup=%v", sc.dedup), func(t *testing.T) { @@ -679,6 +830,39 @@ func TestQuerier_Select(t *testing.T) { } } +func newProxyStore(storeAPIs ...storepb.StoreServer) *store.ProxyStore { + cls := make([]store.Client, len(storeAPIs)) + for i, s := range storeAPIs { + var withoutReplicaLabelsEnabled bool + if srv, ok := s.(*testStoreServer); ok { + withoutReplicaLabelsEnabled = len(srv.respsWithoutReplicaLabels) > 0 + } + cls[i] = &storetestutil.TestClient{ + Name: fmt.Sprintf("%v", i), + StoreClient: storepb.ServerAsClient(s, 0), + MinTime: math.MinInt64, MaxTime: math.MaxInt64, + WithoutReplicaLabelsEnabled: withoutReplicaLabelsEnabled, + } + } + + return store.NewProxyStore( + nil, + nil, + func() []store.Client { return cls }, + component.Query, + nil, + 0, + store.EagerRetrieval, + ) +} + +var emptyLabelsSameAsNotAllocatedLabels = cmp.Transformer("", func(l labels.Labels) labels.Labels { + if len(l) == 0 { + return labels.Labels(nil) + } + return l +}) + func testSelectResponse(t *testing.T, expected []series, res storage.SeriesSet) { var series []storage.Series // Use it as PromQL would do, first gather all series. @@ -686,10 +870,16 @@ func testSelectResponse(t *testing.T, expected []series, res storage.SeriesSet) series = append(series, res.At()) } testutil.Ok(t, res.Err()) - testutil.Equals(t, len(expected), len(series)) + testutil.Equals(t, len(expected), len(series), "got %v series", func() string { + var ret []string + for _, s := range series { + ret = append(ret, s.Labels().String()) + } + return strings.Join(ret, ",") + }()) for i, s := range series { - testutil.Equals(t, expected[i].lset, s.Labels()) + testutil.WithGoCmp(emptyLabelsSameAsNotAllocatedLabels).Equals(t, expected[i].lset, s.Labels()) samples := expandSeries(t, s.Iterator(nil)) expectedCpy := make([]sample, 0, len(expected[i].samples)) for _, s := range expected[i].samples { @@ -701,7 +891,7 @@ func testSelectResponse(t *testing.T, expected []series, res storage.SeriesSet) } expectedCpy = append(expectedCpy, sample{t: s.t, v: v}) } - testutil.Equals(t, expectedCpy, samples, "samples for series %v does not match", i) + testutil.Equals(t, expectedCpy, samples, "samples for series %v does not match", s.Labels()) } } @@ -888,7 +1078,7 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { timeout := 100 * time.Second g := gate.New(2) - q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, s, false, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) + q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, newProxyStore(s), false, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) @@ -958,7 +1148,7 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { timeout := 5 * time.Second g := gate.New(2) - q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, s, true, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) + q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, newProxyStore(s), true, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) @@ -1014,71 +1204,6 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { }) } -func TestSortReplicaLabel(t *testing.T) { - tests := []struct { - input []storepb.Series - exp []storepb.Series - dedupLabels map[string]struct{} - }{ - // 0 Single deduplication label. - { - input: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "3"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-2"}, {Name: "c", Value: "3"}}}, - }, - exp: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-2"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}, {Name: "b", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: "b", Value: "replica-1"}}}, - }, - dedupLabels: map[string]struct{}{"b": {}}, - }, - // 1 Multi deduplication labels. - { - input: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}, {Name: "c", Value: "3"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}, {Name: "c", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-2"}, {Name: "b1", Value: "replica-2"}, {Name: "c", Value: "3"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-2"}, {Name: "c", Value: "3"}}}, - }, - exp: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-2"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-2"}, {Name: "b1", Value: "replica-2"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}}}, - }, - dedupLabels: map[string]struct{}{"b": {}, "b1": {}}, - }, - // Pushdown label at the end. - { - input: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "3"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "4"}, {Name: dedup.PushdownMarker.Name, Value: dedup.PushdownMarker.Value}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-2"}, {Name: "c", Value: "3"}}}, - }, - exp: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-2"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}, {Name: "b", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: dedup.PushdownMarker.Name, Value: dedup.PushdownMarker.Value}, {Name: "b", Value: "replica-1"}}}, - }, - dedupLabels: map[string]struct{}{"b": {}}, - }, - } - for _, test := range tests { - t.Run("", func(t *testing.T) { - sortDedupLabels(test.input, test.dedupLabels) - testutil.Equals(t, test.exp, test.input) - }) - } -} - const hackyStaleMarker = float64(-99999999) func expandSeries(t testing.TB, it chunkenc.Iterator) (res []sample) { @@ -1099,11 +1224,18 @@ type testStoreServer struct { // This field just exist to pseudo-implement the unused methods of the interface. storepb.StoreServer - resps []*storepb.SeriesResponse + resps []*storepb.SeriesResponse + respsWithoutReplicaLabels []*storepb.SeriesResponse } -func (s *testStoreServer) Series(_ *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { - for _, resp := range s.resps { +func (s *testStoreServer) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { + resps := s.resps + + if len(r.WithoutReplicaLabels) > 0 && len(s.respsWithoutReplicaLabels) > 0 { + // If `respsWithoutReplicaLabels` is present, we simulate server that supports without replica label feature. + resps = s.respsWithoutReplicaLabels + } + for _, resp := range resps { err := srv.Send(resp) if err != nil { return err diff --git a/pkg/query/query_bench_test.go b/pkg/query/query_bench_test.go index f72c5fa3bcc..8de127bab56 100644 --- a/pkg/query/query_bench_test.go +++ b/pkg/query/query_bench_test.go @@ -6,9 +6,11 @@ package query import ( "context" "fmt" + "math" "math/rand" "path/filepath" "testing" + "time" "github.com/go-kit/log" "github.com/prometheus/prometheus/model/labels" @@ -81,15 +83,24 @@ func benchQuerySelect(t testutil.TB, totalSamples, totalSeries int, dedup bool) } logger := log.NewNopLogger() - q := &querier{ - ctx: context.Background(), - logger: logger, - proxy: &mockedStoreServer{responses: resps}, - replicaLabels: map[string]struct{}{"a_replica": {}}, - deduplicate: dedup, - selectGate: gate.NewNoop(), - seriesStatsReporter: NoopSeriesStatsReporter, - } + q := newQuerier( + context.Background(), + logger, + math.MinInt64, + math.MaxInt64, + []string{"a_replica"}, + nil, + newProxyStore(&mockedStoreServer{responses: resps}), + dedup, + 0, + false, + false, + false, + gate.NewNoop(), + 10*time.Second, + nil, + NoopSeriesStatsReporter, + ) testSelect(t, q, expectedSeries) } @@ -119,7 +130,8 @@ func testSelect(t testutil.TB, q *querier, expectedSeries []labels.Labels) { t.ResetTimer() for i := 0; i < t.N(); i++ { - ss := q.Select(true, nil) // Select all. + ss := q.Select(true, nil, &labels.Matcher{Value: "foo", Name: "bar", Type: labels.MatchEqual}) + testutil.Ok(t, ss.Err()) testutil.Equals(t, 0, len(ss.Warnings())) if t.IsBenchmark() { @@ -138,22 +150,23 @@ func testSelect(t testutil.TB, q *querier, expectedSeries []labels.Labels) { } testutil.Equals(t, len(expectedSeries), gotSeriesCount) - } else { - // Check more carefully. - var gotSeries []labels.Labels - for ss.Next() { - s := ss.At() - gotSeries = append(gotSeries, s.Labels()) + testutil.Ok(t, ss.Err()) + return + } - // This is when resource usage should actually start growing. - iter := s.Iterator(nil) - for iter.Next() != chunkenc.ValNone { - testT, testV = iter.At() - } - testutil.Ok(t, iter.Err()) + // Check more carefully. + var gotSeries []labels.Labels + for ss.Next() { + s := ss.At() + gotSeries = append(gotSeries, s.Labels()) + + iter := s.Iterator(nil) + for iter.Next() != chunkenc.ValNone { + testT, testV = iter.At() } - testutil.Equals(t, expectedSeries, gotSeries) + testutil.Ok(t, iter.Err()) } + testutil.Equals(t, expectedSeries, gotSeries) testutil.Ok(t, ss.Err()) } }) diff --git a/pkg/query/query_test.go b/pkg/query/query_test.go index 6c68fc5e1ad..35949377e81 100644 --- a/pkg/query/query_test.go +++ b/pkg/query/query_test.go @@ -11,13 +11,13 @@ import ( "testing" "time" + "github.com/efficientgo/core/testutil" "github.com/go-kit/log" "github.com/prometheus/prometheus/storage" - - "github.com/efficientgo/core/testutil" "github.com/thanos-io/thanos/pkg/component" "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/storepb" + storetestutil "github.com/thanos-io/thanos/pkg/store/storepb/testutil" "github.com/thanos-io/thanos/pkg/testutil/custom" ) @@ -49,10 +49,11 @@ func TestQuerier_Proxy(t *testing.T) { testutil.Ok(t, err) // TODO(bwplotka): Parse external labels. - clients = append(clients, inProcessClient{ - t: t, - StoreClient: storepb.ServerAsClient(SelectedStore(store.NewTSDBStore(logger, st.storage.DB, component.Debug, nil), m, st.mint, st.maxt), 0), - name: fmt.Sprintf("store number %v", i), + clients = append(clients, &storetestutil.TestClient{ + Name: fmt.Sprintf("store number %v", i), + StoreClient: storepb.ServerAsClient(selectedStore(store.NewTSDBStore(logger, st.storage.DB, component.Debug, nil), m, st.mint, st.maxt), 0), + MinTime: st.mint, + MaxTime: st.maxt, }) } return q(true, @@ -78,17 +79,17 @@ func TestQuerier_Proxy(t *testing.T) { }) } -// SelectStore allows wrapping another storeAPI with additional time and matcher selection. -type SelectStore struct { +// selectStore allows wrapping another storeEndpoints with additional time and matcher selection. +type selectStore struct { matchers []storepb.LabelMatcher storepb.StoreServer mint, maxt int64 } -// SelectedStore wraps given store with SelectStore. -func SelectedStore(wrapped storepb.StoreServer, matchers []storepb.LabelMatcher, mint, maxt int64) *SelectStore { - return &SelectStore{ +// selectedStore wraps given store with selectStore. +func selectedStore(wrapped storepb.StoreServer, matchers []storepb.LabelMatcher, mint, maxt int64) *selectStore { + return &selectStore{ StoreServer: wrapped, matchers: matchers, mint: mint, @@ -96,7 +97,7 @@ func SelectedStore(wrapped storepb.StoreServer, matchers []storepb.LabelMatcher, } } -func (s *SelectStore) Info(ctx context.Context, r *storepb.InfoRequest) (*storepb.InfoResponse, error) { +func (s *selectStore) Info(ctx context.Context, r *storepb.InfoRequest) (*storepb.InfoResponse, error) { resp, err := s.StoreServer.Info(ctx, r) if err != nil { return nil, err @@ -111,7 +112,7 @@ func (s *SelectStore) Info(ctx context.Context, r *storepb.InfoRequest) (*storep return resp, nil } -func (s *SelectStore) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { +func (s *selectStore) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { if r.MinTime < s.mint { r.MinTime = s.mint } diff --git a/pkg/query/test.go b/pkg/query/test_test.go similarity index 93% rename from pkg/query/test.go rename to pkg/query/test_test.go index a4ccf149348..b9408534ca6 100644 --- a/pkg/query/test.go +++ b/pkg/query/test_test.go @@ -23,10 +23,6 @@ import ( "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/teststorage" - - "github.com/efficientgo/core/testutil" - "github.com/thanos-io/thanos/pkg/store" - "github.com/thanos-io/thanos/pkg/store/storepb" ) var ( @@ -624,39 +620,3 @@ type clearCmd struct{} func (cmd clearCmd) String() string { return "clear" } - -type inProcessClient struct { - t testing.TB - - name string - - storepb.StoreClient - extLset labels.Labels -} - -func NewInProcessClient(t testing.TB, name string, client storepb.StoreClient, extLset labels.Labels) store.Client { - return inProcessClient{ - t: t, - name: name, - StoreClient: client, - extLset: extLset, - } -} - -func (i inProcessClient) LabelSets() []labels.Labels { - return []labels.Labels{i.extLset} -} - -func (i inProcessClient) TimeRange() (mint, maxt int64) { - r, err := i.Info(context.TODO(), &storepb.InfoRequest{}) - testutil.Ok(i.t, err) - return r.MinTime, r.MaxTime -} - -func (i inProcessClient) SupportsSharding() bool { - return false -} - -func (i inProcessClient) SendsSortedSeries() bool { return false } -func (i inProcessClient) String() string { return i.name } -func (i inProcessClient) Addr() (string, bool) { return i.name, true } diff --git a/pkg/receive/multitsdb.go b/pkg/receive/multitsdb.go index 2200822abb7..97d3caf0dc3 100644 --- a/pkg/receive/multitsdb.go +++ b/pkg/receive/multitsdb.go @@ -116,7 +116,7 @@ func (l *localClient) TimeRange() (mint int64, maxt int64) { func (l *localClient) String() string { mint, maxt := l.timeRangeFunc() return fmt.Sprintf( - "LabelSets: %v Mint: %d Maxt: %d", + "LabelSets: %v MinTime: %d MaxTime: %d", labelpb.PromLabelSetsToString(l.LabelSets()), mint, maxt, ) } @@ -129,7 +129,7 @@ func (l *localClient) SupportsSharding() bool { return true } -func (l *localClient) SendsSortedSeries() bool { +func (l *localClient) SupportsWithoutReplicaLabels() bool { return true } diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 23517b8d91c..2315c0a00b9 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -867,16 +867,22 @@ func newBlockSeriesClient( calculateChunkHash bool, batchSize int, chunkFetchDuration prometheus.Histogram, + extLsetToRemove map[string]struct{}, ) *blockSeriesClient { var chunkr *bucketChunkReader if !req.SkipChunks { chunkr = b.chunkReader() } + extLset := b.extLset + if extLsetToRemove != nil { + extLset = rmLabels(extLset.Copy(), extLsetToRemove) + } + return &blockSeriesClient{ ctx: ctx, logger: logger, - extLset: b.extLset, + extLset: extLset, mint: req.MinTime, maxt: req.MaxTime, indexr: b.indexReader(), @@ -1203,6 +1209,14 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie } } + var extLsetToRemove map[string]struct{} + if len(req.WithoutReplicaLabels) > 0 { + extLsetToRemove = make(map[string]struct{}) + for _, l := range req.WithoutReplicaLabels { + extLsetToRemove[l] = struct{}{} + } + } + s.mtx.RLock() for _, bs := range s.blockSets { blockMatchers, ok := bs.labelMatchers(matchers...) @@ -1238,6 +1252,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie s.enableChunkHashCalculation, s.seriesBatchSize, s.metrics.chunkFetchDuration, + extLsetToRemove, ) defer blockClient.Close() @@ -1484,7 +1499,19 @@ func (s *BucketStore) LabelNames(ctx context.Context, req *storepb.LabelNamesReq MaxTime: req.End, SkipChunks: true, } - blockClient := newBlockSeriesClient(newCtx, s.logger, b, seriesReq, nil, bytesLimiter, nil, true, SeriesBatchSize, s.metrics.chunkFetchDuration) + blockClient := newBlockSeriesClient( + newCtx, + s.logger, + b, + seriesReq, + nil, + bytesLimiter, + nil, + true, + SeriesBatchSize, + s.metrics.chunkFetchDuration, + nil, + ) defer blockClient.Close() if err := blockClient.ExpandPostings( @@ -1659,7 +1686,19 @@ func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesR MaxTime: req.End, SkipChunks: true, } - blockClient := newBlockSeriesClient(newCtx, s.logger, b, seriesReq, nil, bytesLimiter, nil, true, SeriesBatchSize, s.metrics.chunkFetchDuration) + blockClient := newBlockSeriesClient( + newCtx, + s.logger, + b, + seriesReq, + nil, + bytesLimiter, + nil, + true, + SeriesBatchSize, + s.metrics.chunkFetchDuration, + nil, + ) defer blockClient.Close() if err := blockClient.ExpandPostings( diff --git a/pkg/store/bucket_e2e_test.go b/pkg/store/bucket_e2e_test.go index e24a83ac3ed..ccea0ff92af 100644 --- a/pkg/store/bucket_e2e_test.go +++ b/pkg/store/bucket_e2e_test.go @@ -244,7 +244,6 @@ func gatherFamily(t testing.TB, reg prometheus.Gatherer, familyName string) *dto return nil } -// TODO(bwplotka): Benchmark Series. func testBucketStore_e2e(t *testing.T, ctx context.Context, s *storeSuite) { t.Helper() @@ -286,6 +285,27 @@ func testBucketStore_e2e(t *testing.T, ctx context.Context, s *storeSuite) { {{Name: "a", Value: "2"}, {Name: "c", Value: "2"}, {Name: "ext2", Value: "value2"}}, }, }, + { + req: &storepb.SeriesRequest{ + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_RE, Name: "a", Value: "1|2"}, + }, + MinTime: mint, + MaxTime: maxt, + WithoutReplicaLabels: []string{"ext1", "ext2"}, + }, + expectedChunkLen: 3, + expected: [][]labelpb.ZLabel{ + {{Name: "a", Value: "1"}, {Name: "b", Value: "1"}}, + {{Name: "a", Value: "1"}, {Name: "b", Value: "2"}}, + {{Name: "a", Value: "1"}, {Name: "c", Value: "1"}}, + {{Name: "a", Value: "1"}, {Name: "c", Value: "2"}}, + {{Name: "a", Value: "2"}, {Name: "b", Value: "1"}}, + {{Name: "a", Value: "2"}, {Name: "b", Value: "2"}}, + {{Name: "a", Value: "2"}, {Name: "c", Value: "1"}}, + {{Name: "a", Value: "2"}, {Name: "c", Value: "2"}}, + }, + }, { req: &storepb.SeriesRequest{ Matchers: []storepb.LabelMatcher{ diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index c115be4d6b7..a3b42b963a8 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -2141,7 +2141,7 @@ func TestLabelNamesAndValuesHints(t *testing.T) { } } -func TestSeries_ChuncksHaveHashRepresentation(t *testing.T) { +func TestSeries_ChunksHaveHashRepresentation(t *testing.T) { tb := testutil.NewTB(t) tmpDir := t.TempDir() @@ -2441,7 +2441,19 @@ func benchmarkBlockSeriesWithConcurrency(b *testing.B, concurrency int, blockMet testutil.Ok(b, err) dummyHistogram := prometheus.NewHistogram(prometheus.HistogramOpts{}) - blockClient := newBlockSeriesClient(ctx, nil, blk, req, chunksLimiter, NewBytesLimiterFactory(0)(nil), nil, false, SeriesBatchSize, dummyHistogram) + blockClient := newBlockSeriesClient( + ctx, + nil, + blk, + req, + chunksLimiter, + NewBytesLimiterFactory(0)(nil), + nil, + false, + SeriesBatchSize, + dummyHistogram, + nil, + ) testutil.Ok(b, blockClient.ExpandPostings(matchers, seriesLimiter)) defer blockClient.Close() diff --git a/pkg/store/local.go b/pkg/store/local.go index 5cc20ee4600..4e88c0a7e3f 100644 --- a/pkg/store/local.go +++ b/pkg/store/local.go @@ -182,7 +182,8 @@ func (s *LocalStore) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesSe chosen = chosen[:0] resp := &storepb.Series{ - Labels: series.Labels, + // Copy labels as in-process clients like proxy tend to work on same memory for labels. + Labels: labelpb.DeepCopy(series.Labels), Chunks: make([]storepb.AggrChunk, 0, len(s.sortedChunks[si])), } diff --git a/pkg/store/prometheus_test.go b/pkg/store/prometheus_test.go index f77504eb1ec..eb9d80ed159 100644 --- a/pkg/store/prometheus_test.go +++ b/pkg/store/prometheus_test.go @@ -67,7 +67,7 @@ func testPrometheusStoreSeriesE2e(t *testing.T, prefix string) { limitMinT := int64(0) proxy, err := NewPrometheusStore(nil, nil, promclient.NewDefaultClient(), u, component.Sidecar, func() labels.Labels { return labels.FromStrings("region", "eu-west") }, - func() (int64, int64) { return limitMinT, -1 }, nil) // Maxt does not matter. + func() (int64, int64) { return limitMinT, -1 }, nil) // MaxTime does not matter. testutil.Ok(t, err) // Query all three samples except for the first one. Since we round up queried data diff --git a/pkg/store/proxy.go b/pkg/store/proxy.go index 79dc0ab4bb3..08c4e06a2e6 100644 --- a/pkg/store/proxy.go +++ b/pkg/store/proxy.go @@ -53,11 +53,9 @@ type Client interface { // SupportsSharding returns true if sharding is supported by the underlying store. SupportsSharding() bool - // SendsSortedSeries returns true if the underlying store sends series sorded by - // their labels. - // The field can be used to indicate to the querier whether it needs to sort - // received series before deduplication. - SendsSortedSeries() bool + // SupportsWithoutReplicaLabels returns true if trimming replica labels + // and sorted response is supported by the underlying store. + SupportsWithoutReplicaLabels() bool // String returns the string representation of the store client. String() string @@ -268,6 +266,7 @@ func (s *ProxyStore) Series(originalRequest *storepb.SeriesRequest, srv storepb. PartialResponseDisabled: originalRequest.PartialResponseDisabled, PartialResponseStrategy: originalRequest.PartialResponseStrategy, ShardInfo: originalRequest.ShardInfo, + WithoutReplicaLabels: originalRequest.WithoutReplicaLabels, } stores := []Client{} @@ -293,7 +292,7 @@ func (s *ProxyStore) Series(originalRequest *storepb.SeriesRequest, srv storepb. storeDebugMsgs = append(storeDebugMsgs, fmt.Sprintf("store %s queried", st)) - respSet, err := newAsyncRespSet(srv.Context(), st, r, s.responseTimeout, s.retrievalStrategy, st.SupportsSharding(), &s.buffers, r.ShardInfo, reqLogger, s.metrics.emptyStreamResponses) + respSet, err := newAsyncRespSet(srv.Context(), st, r, s.responseTimeout, s.retrievalStrategy, &s.buffers, r.ShardInfo, reqLogger, s.metrics.emptyStreamResponses) if err != nil { level.Error(reqLogger).Log("err", err) @@ -384,7 +383,7 @@ func labelSetsMatch(matchers []*labels.Matcher, lset ...labels.Labels) bool { for _, ls := range lset { notMatched := false for _, m := range matchers { - if lv := ls.Get(m.Name); lv != "" && !m.Matches(lv) { + if lv := ls.Get(m.Name); ls.Has(m.Name) && !m.Matches(lv) { notMatched = true break } diff --git a/pkg/store/proxy_heap.go b/pkg/store/proxy_heap.go index 5cdb5a0b781..028ac81a7ec 100644 --- a/pkg/store/proxy_heap.go +++ b/pkg/store/proxy_heap.go @@ -12,9 +12,8 @@ import ( "sync" "time" - "github.com/go-kit/log" - "github.com/cespare/xxhash/v2" + "github.com/go-kit/log" "github.com/go-kit/log/level" grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tracing" "github.com/opentracing/opentracing-go" @@ -27,104 +26,93 @@ import ( "github.com/thanos-io/thanos/pkg/tracing" ) -// dedupResponseHeap is a wrapper around ProxyResponseHeap -// that removes duplicated identical chunks identified by the same labelset and checksum. -// It uses a hashing function to do that. type dedupResponseHeap struct { h *ProxyResponseHeap - responses []*storepb.SeriesResponse + bufferedSameSeries []*storepb.SeriesResponse - previousResponse *storepb.SeriesResponse - previousNext bool + bufferedResp []*storepb.SeriesResponse + buffRespI int + + prev *storepb.SeriesResponse + ok bool } +// NewDedupResponseHeap returns a wrapper around ProxyResponseHeap that merged duplicated series messages into one. +// It also deduplicates identical chunks identified by the same checksum from each series message. func NewDedupResponseHeap(h *ProxyResponseHeap) *dedupResponseHeap { + ok := h.Next() + var prev *storepb.SeriesResponse + if ok { + prev = h.At() + } return &dedupResponseHeap{ - h: h, - previousNext: h.Next(), + h: h, + ok: ok, + prev: prev, } } func (d *dedupResponseHeap) Next() bool { - d.responses = d.responses[:0] - - // If there is something buffered that is *not* a series. - if d.previousResponse != nil && d.previousResponse.GetSeries() == nil { - d.responses = append(d.responses, d.previousResponse) - d.previousResponse = nil - d.previousNext = d.h.Next() - return len(d.responses) > 0 || d.previousNext + if d.buffRespI+1 < len(d.bufferedResp) { + d.buffRespI++ + return true } - var resp *storepb.SeriesResponse - var nextHeap bool - - // If buffered then use it. - if d.previousResponse != nil { - resp = d.previousResponse - d.previousResponse = nil - } else { - // If not buffered then check whether there is anything. - nextHeap = d.h.Next() - if !nextHeap { - return false - } - resp = d.h.At() + if !d.ok && d.prev == nil { + return false } - // Append buffered or retrieved response. - d.responses = append(d.responses, resp) - - // Update previousNext. - defer func(next *bool) { - d.previousNext = *next - }(&nextHeap) - - if resp.GetSeries() == nil { - return len(d.responses) > 0 || d.previousNext - } + d.buffRespI = 0 + d.bufferedResp = d.bufferedResp[:0] + d.bufferedSameSeries = d.bufferedSameSeries[:0] + var s *storepb.SeriesResponse for { - nextHeap = d.h.Next() - if !nextHeap { - break + if d.prev == nil { + d.ok = d.h.Next() + if !d.ok { + if len(d.bufferedSameSeries) > 0 { + d.bufferedResp = append(d.bufferedResp, chainSeriesAndRemIdenticalChunks(d.bufferedSameSeries)) + } + return len(d.bufferedResp) > 0 + } + s = d.h.At() + } else { + s = d.prev + d.prev = nil } - resp = d.h.At() - if resp.GetSeries() == nil { - d.previousResponse = resp - break + + if s.GetSeries() == nil { + d.bufferedResp = append(d.bufferedResp, s) + continue } - lbls := resp.GetSeries().Labels - lastLbls := d.responses[len(d.responses)-1].GetSeries().Labels + if len(d.bufferedSameSeries) == 0 { + d.bufferedSameSeries = append(d.bufferedSameSeries, s) + continue + } - if labels.Compare(labelpb.ZLabelsToPromLabels(lbls), labelpb.ZLabelsToPromLabels(lastLbls)) == 0 { - d.responses = append(d.responses, resp) - } else { - // This one is different. It will be taken care of via the next Next() call. - d.previousResponse = resp - break + lbls := d.bufferedSameSeries[0].GetSeries().Labels + atLbls := s.GetSeries().Labels + + if labels.Compare(labelpb.ZLabelsToPromLabels(lbls), labelpb.ZLabelsToPromLabels(atLbls)) == 0 { + d.bufferedSameSeries = append(d.bufferedSameSeries, s) + continue } - } - return len(d.responses) > 0 || d.previousNext -} + d.bufferedResp = append(d.bufferedResp, chainSeriesAndRemIdenticalChunks(d.bufferedSameSeries)) + d.prev = s -func (d *dedupResponseHeap) At() *storepb.SeriesResponse { - if len(d.responses) == 0 { - panic("BUG: At() called with no responses; please call At() only if Next() returns true") - } else if len(d.responses) == 1 { - return d.responses[0] + return true } +} +func chainSeriesAndRemIdenticalChunks(series []*storepb.SeriesResponse) *storepb.SeriesResponse { chunkDedupMap := map[uint64]*storepb.AggrChunk{} - for _, resp := range d.responses { - if resp.GetSeries() == nil { - continue - } - for _, chk := range resp.GetSeries().Chunks { + for _, s := range series { + for _, chk := range s.GetSeries().Chunks { for _, field := range []*storepb.Chunk{ chk.Raw, chk.Count, chk.Max, chk.Min, chk.Sum, chk.Counter, } { @@ -147,7 +135,7 @@ func (d *dedupResponseHeap) At() *storepb.SeriesResponse { // If no chunks were requested. if len(chunkDedupMap) == 0 { - return d.responses[0] + return series[0] } finalChunks := make([]storepb.AggrChunk, 0, len(chunkDedupMap)) @@ -159,16 +147,16 @@ func (d *dedupResponseHeap) At() *storepb.SeriesResponse { return finalChunks[i].Compare(finalChunks[j]) > 0 }) - // Guaranteed to be a series because Next() only buffers one - // warning at a time that gets handled in the beginning. - lbls := d.responses[0].GetSeries().Labels - return storepb.NewSeriesResponse(&storepb.Series{ - Labels: lbls, + Labels: series[0].GetSeries().Labels, Chunks: finalChunks, }) } +func (d *dedupResponseHeap) At() *storepb.SeriesResponse { + return d.bufferedResp[d.buffRespI] +} + // ProxyResponseHeap is a heap for storepb.SeriesSets. // It performs k-way merge between all of those sets. // TODO(GiedriusS): can be improved with a tournament tree. @@ -225,6 +213,8 @@ type ProxyResponseHeapNode struct { rs respSet } +// NewProxyResponseHeap returns heap that k-way merge series together. +// It's agnostic to duplicates and overlaps, it forwards all duplicated series in random order. func NewProxyResponseHeap(seriesSets ...respSet) *ProxyResponseHeap { ret := make(ProxyResponseHeap, 0, len(seriesSets)) @@ -431,6 +421,7 @@ func newLazyRespSet( } if err != nil { + // TODO(bwplotka): Return early on error. Don't wait of dedup, merge and sort if partial response is disabled. var rerr error if t != nil && !t.Stop() && errors.Is(err, context.Canceled) { // Most likely the per-Recv timeout has been reached. @@ -490,24 +481,26 @@ func newLazyRespSet( type RetrievalStrategy string const ( + // LazyRetrieval allows readers (e.g. PromQL engine) to use (stream) data as soon as possible. LazyRetrieval RetrievalStrategy = "lazy" - - // TODO(GiedriusS): remove eager retrieval once - // https://github.com/prometheus/prometheus/blob/ce6a643ee88fba7c02fbd0459c4d0ac498f512dd/promql/engine.go#L877-L902 - // is removed. + // EagerRetrieval is optimized to read all into internal buffer before returning to readers (e.g. PromQL engine). + // This currently preferred because: + // * Both PromQL engines (old and new) want all series ASAP to make decisions. + // * Querier buffers all responses when using StoreAPI internally. EagerRetrieval RetrievalStrategy = "eager" ) -func newAsyncRespSet(ctx context.Context, +func newAsyncRespSet( + ctx context.Context, st Client, req *storepb.SeriesRequest, frameTimeout time.Duration, retrievalStrategy RetrievalStrategy, - storeSupportsSharding bool, buffers *sync.Pool, shardInfo *storepb.ShardInfo, logger log.Logger, - emptyStreamResponses prometheus.Counter) (respSet, error) { + emptyStreamResponses prometheus.Counter, +) (respSet, error) { var span opentracing.Span var closeSeries context.CancelFunc @@ -532,10 +525,9 @@ func newAsyncRespSet(ctx context.Context, shardMatcher := shardInfo.Matcher(buffers) - applySharding := shardInfo != nil && !storeSupportsSharding + applySharding := shardInfo != nil && !st.SupportsSharding() if applySharding { - msg := "Applying series sharding in the proxy since there is not support in the underlying store" - level.Debug(logger).Log("msg", msg, "store", st.String()) + level.Debug(logger).Log("msg", "Applying series sharding in the proxy since there is not support in the underlying store", "store", st.String()) } cl, err := st.Series(seriesCtx, req) @@ -548,6 +540,18 @@ func newAsyncRespSet(ctx context.Context, return nil, err } + var labelsToRemove map[string]struct{} + if !st.SupportsWithoutReplicaLabels() && len(req.WithoutReplicaLabels) > 0 { + level.Warn(logger).Log("msg", "detecting store that does not support without replica label setting. "+ + "Falling back to eager retrieval with additional sort. Make sure your storeAPI supports it to speed up your queries", "store", st.String()) + retrievalStrategy = EagerRetrieval + + labelsToRemove = make(map[string]struct{}) + for _, replicaLabel := range req.WithoutReplicaLabels { + labelsToRemove[replicaLabel] = struct{}{} + } + } + switch retrievalStrategy { case LazyRetrieval: return newLazyRespSet( @@ -573,6 +577,7 @@ func newAsyncRespSet(ctx context.Context, shardMatcher, applySharding, emptyStreamResponses, + labelsToRemove, ), nil default: panic(fmt.Sprintf("unsupported retrieval strategy %s", retrievalStrategy)) @@ -592,6 +597,7 @@ func (l *lazyRespSet) Close() { // eagerRespSet is a SeriesSet that blocks until all data is retrieved from // the StoreAPI. +// NOTE(bwplotka): It also resorts the series (and emits warning) if the client.SupportsWithoutReplicaLabels() is false. type eagerRespSet struct { // Generic parameters. span opentracing.Span @@ -603,6 +609,7 @@ type eagerRespSet struct { frameTimeout time.Duration shardMatcher *storepb.ShardMatcher + removeLabels map[string]struct{} // Internal bookkeeping. bufferedResponses []*storepb.SeriesResponse @@ -620,6 +627,7 @@ func newEagerRespSet( shardMatcher *storepb.ShardMatcher, applySharding bool, emptyStreamResponses prometheus.Counter, + removeLabels map[string]struct{}, ) respSet { ret := &eagerRespSet{ span: span, @@ -631,6 +639,7 @@ func newEagerRespSet( bufferedResponses: []*storepb.SeriesResponse{}, wg: &sync.WaitGroup{}, shardMatcher: shardMatcher, + removeLabels: removeLabels, } ret.wg.Add(1) @@ -656,6 +665,8 @@ func newEagerRespSet( } }() + // TODO(bwplotka): Consider improving readability by getting rid of anonymous functions and merging eager and + // lazyResponse into one struct. handleRecvResponse := func(t *time.Timer) bool { if t != nil { defer t.Reset(frameTimeout) @@ -673,6 +684,7 @@ func newEagerRespSet( return false } if err != nil { + // TODO(bwplotka): Return early on error. Don't wait of dedup, merge and sort if partial response is disabled. var rerr error if t != nil && !t.Stop() && errors.Is(err, context.Canceled) { // Most likely the per-Recv timeout has been reached. @@ -710,14 +722,59 @@ func newEagerRespSet( for { if !handleRecvResponse(t) { - return + break } } + + // This should be used only for stores that does not support doing this on server side. + // See docs/proposals-accepted/20221129-avoid-global-sort.md for details. + if len(l.removeLabels) > 0 { + sortWithoutLabels(l.bufferedResponses, l.removeLabels) + } + }(st, ret) return ret } +func rmLabels(l labels.Labels, labelsToRemove map[string]struct{}) labels.Labels { + for i := 0; i < len(l); i++ { + if _, ok := labelsToRemove[l[i].Name]; !ok { + continue + } + l = append(l[:i], l[i+1:]...) + i-- + } + return l +} + +// sortWithoutLabels removes given labels from series and re-sorts the series responses that the same +// series with different labels are coming right after each other. Other types of responses are moved to front. +func sortWithoutLabels(set []*storepb.SeriesResponse, labelsToRemove map[string]struct{}) { + for _, s := range set { + ser := s.GetSeries() + if ser == nil { + continue + } + + ser.Labels = labelpb.ZLabelsFromPromLabels(rmLabels(labelpb.ZLabelsToPromLabels(ser.Labels), labelsToRemove)) + } + + // With the re-ordered label sets, re-sorting all series aligns the same series + // from different replicas sequentially. + sort.Slice(set, func(i, j int) bool { + si := set[i].GetSeries() + if si == nil { + return true + } + sj := set[j].GetSeries() + if sj == nil { + return false + } + return labels.Compare(labelpb.ZLabelsToPromLabels(si.Labels), labelpb.ZLabelsToPromLabels(sj.Labels)) < 0 + }) +} + func (l *eagerRespSet) Close() { l.shardMatcher.Close() } diff --git a/pkg/store/proxy_heap_test.go b/pkg/store/proxy_heap_test.go new file mode 100644 index 00000000000..6616db40165 --- /dev/null +++ b/pkg/store/proxy_heap_test.go @@ -0,0 +1,146 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package store + +import ( + "testing" + + "github.com/efficientgo/core/testutil" + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/thanos/pkg/dedup" + "github.com/thanos-io/thanos/pkg/errors" + "github.com/thanos-io/thanos/pkg/store/storepb" +) + +func TestSortWithoutLabels(t *testing.T) { + for _, tcase := range []struct { + input []*storepb.SeriesResponse + exp []*storepb.SeriesResponse + dedupLabels map[string]struct{} + }{ + // Single deduplication label. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-2", "c", "3")), + }, + exp: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "4")), + }, + dedupLabels: map[string]struct{}{"b": {}}, + }, + // Multi deduplication labels. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "b1", "replica-1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "b1", "replica-1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "b1", "replica-1", "c", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "b1", "replica-2", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-2", "c", "3")), + }, + exp: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "4")), + }, + dedupLabels: map[string]struct{}{"b": {}, "b1": {}}, + }, + // Pushdown label at the end. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "4", dedup.PushdownMarker.Name, dedup.PushdownMarker.Value)), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-2", "c", "3")), + }, + exp: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "4", dedup.PushdownMarker.Name, dedup.PushdownMarker.Value)), + }, + dedupLabels: map[string]struct{}{"b": {}}, + }, + // Non series responses mixed. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3", "d", "4")), + storepb.NewWarnSeriesResponse(errors.Newf("yolo")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-2", "c", "3")), + }, + exp: []*storepb.SeriesResponse{ + storepb.NewWarnSeriesResponse(errors.Newf("yolo")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "4")), + }, + dedupLabels: map[string]struct{}{"b": {}}, + }, + // Longer series. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings( + "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", + "gprd", "fqdn", "web-08-sv-gprd.c.gitlab-production.internal", "instance", "web-08-sv-gprd.c.gitlab-production.internal:8083", "job", "gitlab-rails", "monitor", "app", "provider", + "gcp", "region", "us-east", "replica", "01", "shard", "default", "stage", "main", "tier", "sv", "type", "web", + )), + }, + exp: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings( + // No replica label anymore. + "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", + "gprd", "fqdn", "web-08-sv-gprd.c.gitlab-production.internal", "instance", "web-08-sv-gprd.c.gitlab-production.internal:8083", "job", "gitlab-rails", "monitor", "app", "provider", + "gcp", "region", "us-east", "shard", "default", "stage", "main", "tier", "sv", "type", "web", + )), + }, + dedupLabels: map[string]struct{}{"replica": {}}, + }, + } { + t.Run("", func(t *testing.T) { + sortWithoutLabels(tcase.input, tcase.dedupLabels) + testutil.Equals(t, tcase.exp, tcase.input) + }) + } +} + +// labelsFromStrings is like labels.FromString, but it does not sort the input. +func labelsFromStrings(ss ...string) labels.Labels { + if len(ss)%2 != 0 { + panic("invalid number of strings") + } + res := make(labels.Labels, 0, len(ss)/2) + for i := 0; i < len(ss); i += 2 { + res = append(res, labels.Label{Name: ss[i], Value: ss[i+1]}) + } + + return res +} + +func BenchmarkSortWithoutLabels(b *testing.B) { + resps := make([]*storepb.SeriesResponse, 1e4) + labelsToRemove := map[string]struct{}{ + "a": {}, "b": {}, + } + + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + for i := 0; i < 1e4; i++ { + resps[i] = storeSeriesResponse(b, labels.FromStrings("a", "1", "b", "replica-1", "c", "replica-1", "d", "1")) + } + b.StartTimer() + sortWithoutLabels(resps, labelsToRemove) + } +} diff --git a/pkg/store/proxy_test.go b/pkg/store/proxy_test.go index eb852d67c6a..61d9bc1965d 100644 --- a/pkg/store/proxy_test.go +++ b/pkg/store/proxy_test.go @@ -36,41 +36,6 @@ import ( "github.com/thanos-io/thanos/pkg/testutil/custom" ) -type testClient struct { - // Just to pass interface check. - storepb.StoreClient - - labelSets []labels.Labels - minTime int64 - maxTime int64 - supportsSharding bool - isLocalStore bool -} - -func (c testClient) LabelSets() []labels.Labels { - return c.labelSets -} - -func (c testClient) TimeRange() (int64, int64) { - return c.minTime, c.maxTime -} - -func (c testClient) SupportsSharding() bool { - return c.supportsSharding -} - -func (c testClient) SendsSortedSeries() bool { - return false -} - -func (c testClient) String() string { - return "test" -} - -func (c testClient) Addr() (string, bool) { - return "testaddr", c.isLocalStore -} - type mockedSeriesServer struct { storepb.Store_SeriesServer ctx context.Context @@ -138,14 +103,14 @@ func TestProxyStore_Series(t *testing.T) { { title: "no storeAPI available for 301-302 time range", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -158,15 +123,15 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; no series for ext=2 external label matcher", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, }, req: &storepb.SeriesRequest{ @@ -179,15 +144,15 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for ext=1 external label matcher", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, }, req: &storepb.SeriesRequest{ @@ -205,14 +170,14 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for any external label matcher", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{4, 3}}, []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -223,21 +188,21 @@ func TestProxyStore_Series(t *testing.T) { expectedSeries: []rawSeries{ { lset: labels.FromStrings("a", "a"), - chunks: [][]sample{{{4, 3}}, {{0, 0}, {2, 1}, {3, 2}}}, // No sort merge. + chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{4, 3}}}, }, }, }, { title: "storeAPI available for time range; available series for any external label matcher, but selector blocks", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, }, selectorLabels: labels.FromStrings("ext", "2"), @@ -250,14 +215,14 @@ func TestProxyStore_Series(t *testing.T) { { title: "no validation if storeAPI follow matching contract", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -276,7 +241,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "complex scenario with storeAPIs warnings", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}, []sample{{4, 3}}), @@ -285,45 +250,45 @@ func TestProxyStore_Series(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{2, 2}, {3, 3}, {4, 4}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{100, 1}, {300, 3}, {400, 4}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "outside"), []sample{{1, 1}}), }, }, // Outside range for store itself. - minTime: 301, - maxTime: 302, + MinTime: 301, + MaxTime: 302, }, }, req: &storepb.SeriesRequest{ @@ -347,28 +312,114 @@ func TestProxyStore_Series(t *testing.T) { }, expectedWarningsLen: 2, }, + { + title: "storeAPI available for time range; available two duplicated series for ext=1 external label matcher from 2 storeAPIs", + storeAPIs: []Client{ + &storetestutil.TestClient{ + StoreClient: &mockedStoreAPI{ + RespSeries: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), + }, + }, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + }, + &storetestutil.TestClient{ + StoreClient: &mockedStoreAPI{ + RespSeries: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{1, 4}, {2, 5}, {3, 6}}), + }, + }, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + }, + }, + req: &storepb.SeriesRequest{ + MinTime: 1, + MaxTime: 300, + Matchers: []storepb.LabelMatcher{{Name: "ext", Value: "1", Type: storepb.LabelMatcher_EQ}}, + }, + expectedSeries: []rawSeries{ + { + lset: labels.FromStrings("a", "a"), + chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{1, 4}, {2, 5}, {3, 6}}}, + }, + }, + }, + { + title: "storeAPI available for time range; available a few duplicated series for ext=1 external label matcher, mixed storeAPIs", + storeAPIs: []Client{ + &storetestutil.TestClient{ + StoreClient: &mockedStoreAPI{ + RespSeries: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "1"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "1"), []sample{{100, 1}, {300, 3}, {400, 4}}), + }, + }, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + }, + &storetestutil.TestClient{ + StoreClient: &mockedStoreAPI{ + RespSeries: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{2, 1}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "2"), []sample{{10, 10}, {30, 30}, {40, 40}}), + }, + }, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + }, + }, + req: &storepb.SeriesRequest{ + MinTime: 1, + MaxTime: 300, + Matchers: []storepb.LabelMatcher{{Name: "ext", Value: "1", Type: storepb.LabelMatcher_EQ}}, + }, + expectedSeries: []rawSeries{ + { + lset: labels.FromStrings("a", "1", "w", "1"), + chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{2, 1}}, {{5, 5}, {6, 6}, {7, 7}}, {{5, 5}, {7, 7}}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "1"), + chunks: [][]sample{{{1, 1}, {2, 2}, {3, 3}}, {{2, 2}, {3, 3}, {4, 4}}, {{100, 1}, {300, 3}, {400, 4}}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "2"), + chunks: [][]sample{{{10, 10}, {30, 30}, {40, 40}}}, + }, + }, + }, { title: "same external labels are validated during upload and on querier storeset, proxy does not care", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 11}, {2, 22}, {3, 33}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -386,24 +437,24 @@ func TestProxyStore_Series(t *testing.T) { { title: "partial response enabled", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -422,24 +473,24 @@ func TestProxyStore_Series(t *testing.T) { { title: "partial response disabled", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -449,20 +500,21 @@ func TestProxyStore_Series(t *testing.T) { PartialResponseDisabled: true, PartialResponseStrategy: storepb.PartialResponseStrategy_ABORT, }, - expectedErr: errors.New("fetch series for {ext=\"1\"} test: error!"), + expectedErr: errors.New("fetch series for {ext=\"1\"} : error!"), }, { title: "storeAPI available for time range; available series for ext=1 external label matcher; allowed by store debug matcher", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + Name: "testaddr", }, }, req: &storepb.SeriesRequest{ @@ -482,15 +534,16 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for ext=1 external label matcher; blocked by store debug matcher.", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + Name: "testaddr", }, }, req: &storepb.SeriesRequest{ @@ -504,7 +557,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "sharded series response", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -512,9 +565,9 @@ func TestProxyStore_Series(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, }, req: &storepb.SeriesRequest{ @@ -540,38 +593,44 @@ func TestProxyStore_Series(t *testing.T) { }, }, } { - for _, strategy := range []RetrievalStrategy{EagerRetrieval, LazyRetrieval} { - if ok := t.Run(fmt.Sprintf("%s/%s", tc.title, strategy), func(t *testing.T) { - q := NewProxyStore(nil, - nil, - func() []Client { return tc.storeAPIs }, - component.Query, - tc.selectorLabels, - 5*time.Second, strategy, - ) - - ctx := context.Background() - if len(tc.storeDebugMatchers) > 0 { - ctx = context.WithValue(ctx, StoreMatcherKey, tc.storeDebugMatchers) - } - - s := newStoreSeriesServer(ctx) - err := q.Series(tc.req, s) - if tc.expectedErr != nil { - testutil.NotOk(t, err) - testutil.Equals(t, tc.expectedErr.Error(), err.Error()) - return - } - testutil.Ok(t, err) - - seriesEquals(t, tc.expectedSeries, s.SeriesSet) - testutil.Equals(t, tc.expectedWarningsLen, len(s.Warnings), "got %v", s.Warnings) - - }); !ok { - return + t.Run(tc.title, func(t *testing.T) { + for _, replicaLabelSupport := range []bool{false, true} { + t.Run(fmt.Sprintf("replica_support=%v", replicaLabelSupport), func(t *testing.T) { + for _, s := range tc.storeAPIs { + cl := s.(*storetestutil.TestClient) + cl.WithoutReplicaLabelsEnabled = replicaLabelSupport + } + for _, strategy := range []RetrievalStrategy{EagerRetrieval, LazyRetrieval} { + t.Run(string(strategy), func(t *testing.T) { + q := NewProxyStore(nil, + nil, + func() []Client { return tc.storeAPIs }, + component.Query, + tc.selectorLabels, + 5*time.Second, strategy, + ) + + ctx := context.Background() + if len(tc.storeDebugMatchers) > 0 { + ctx = context.WithValue(ctx, StoreMatcherKey, tc.storeDebugMatchers) + } + + s := newStoreSeriesServer(ctx) + err := q.Series(tc.req, s) + if tc.expectedErr != nil { + testutil.NotOk(t, err) + testutil.Equals(t, tc.expectedErr.Error(), err.Error()) + return + } + testutil.Ok(t, err) + + seriesEquals(t, tc.expectedSeries, s.SeriesSet) + testutil.Equals(t, tc.expectedWarningsLen, len(s.Warnings), "got %v warnings", s.Warnings) + }) + } + }) } - } - + }) } } @@ -597,7 +656,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st errors out after some delay; 2nd store is fast", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -608,11 +667,11 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { injectedError: errors.New("test"), injectedErrorIndex: 1, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -635,9 +694,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("b", "a"), []sample{{4, 1}, {5, 2}, {6, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -652,7 +711,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is slow, 2nd store is fast;", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -660,20 +719,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -688,18 +747,18 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is fast, 2nd store is slow;", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -707,9 +766,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -724,7 +783,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is slow on 2nd series, 2nd store is fast;", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -735,20 +794,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { RespDuration: 10 * time.Second, SlowSeriesIndex: 2, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -763,7 +822,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is fast to respond, 2nd store is slow on 2nd series;", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -772,11 +831,11 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{6, 1}, {7, 2}, {8, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -785,9 +844,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { RespDuration: 10 * time.Second, SlowSeriesIndex: 2, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -802,7 +861,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is slow to respond, 2nd store is fast;", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -810,20 +869,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("b", "c"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -842,18 +901,18 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is fast, 2nd store is slow;", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -861,9 +920,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -882,18 +941,18 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is fast, 2-3 is slow, 4th is fast;", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -901,11 +960,11 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -913,20 +972,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("d", "f"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -949,7 +1008,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is slow on 2nd series, 2nd store is fast", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -960,20 +1019,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { RespDuration: 10 * time.Second, SlowSeriesIndex: 2, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("b", "c"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -996,7 +1055,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; all stores respond 3s", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1005,9 +1064,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 3 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -1028,7 +1087,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; all stores respond 3s", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1037,11 +1096,11 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 3 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("b", "c"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1050,9 +1109,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 3 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -1127,11 +1186,11 @@ func TestProxyStore_Series_RequestParamsProxied(t *testing.T) { }, } cls := []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: m, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, } q := NewProxyStore(nil, @@ -1167,14 +1226,14 @@ func TestProxyStore_Series_RegressionFillResponseChannel(t *testing.T) { var cls []Client for i := 0; i < 10; i++ { - cls = append(cls, &testClient{ + cls = append(cls, &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("test error"), }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }) - cls = append(cls, &testClient{ + cls = append(cls, &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -1189,8 +1248,8 @@ func TestProxyStore_Series_RegressionFillResponseChannel(t *testing.T) { storepb.NewWarnSeriesResponse(errors.New("warning")), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }) } @@ -1228,18 +1287,18 @@ func TestProxyStore_LabelValues(t *testing.T) { }, } cls := []Client{ - &testClient{StoreClient: m1}, - &testClient{StoreClient: &mockedStoreAPI{ + &storetestutil.TestClient{StoreClient: m1}, + &storetestutil.TestClient{StoreClient: &mockedStoreAPI{ RespLabelValues: &storepb.LabelValuesResponse{ Values: []string{"3", "4"}, }, }}, - &testClient{StoreClient: &mockedStoreAPI{ + &storetestutil.TestClient{StoreClient: &mockedStoreAPI{ RespLabelValues: &storepb.LabelValuesResponse{ Values: []string{"5", "6"}, }}, - minTime: timestamp.FromTime(time.Now().Add(-1 * time.Minute)), - maxTime: timestamp.FromTime(time.Now()), + MinTime: timestamp.FromTime(time.Now().Add(-1 * time.Minute)), + MaxTime: timestamp.FromTime(time.Now()), }, } q := NewProxyStore(nil, @@ -1296,14 +1355,14 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response disabled", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "c", "d"}, @@ -1322,17 +1381,18 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response disabled, but returns error", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, + Name: "test", }, }, req: &storepb.LabelNamesRequest{ @@ -1345,14 +1405,14 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response enabled", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, @@ -1369,23 +1429,23 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "stores filtered by time range", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, - minTime: timestamp.FromTime(time.Now().Add(-4 * time.Hour)), - maxTime: timestamp.FromTime(time.Now().Add(-3 * time.Hour)), + MinTime: timestamp.FromTime(time.Now().Add(-4 * time.Hour)), + MaxTime: timestamp.FromTime(time.Now().Add(-3 * time.Hour)), }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"c", "d"}, }, }, - minTime: timestamp.FromTime(time.Now().Add(-2 * time.Hour)), - maxTime: timestamp.FromTime(time.Now().Add(-1 * time.Hour)), + MinTime: timestamp.FromTime(time.Now().Add(-2 * time.Hour)), + MaxTime: timestamp.FromTime(time.Now().Add(-1 * time.Hour)), }, }, req: &storepb.LabelNamesRequest{ @@ -1399,12 +1459,13 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "store matchers blocks", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, + Name: "testaddr", }, }, req: &storepb.LabelNamesRequest{ @@ -1419,12 +1480,13 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "store matchers allows", storeAPIs: []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, + Name: "testaddr", }, }, req: &storepb.LabelNamesRequest{ @@ -1475,26 +1537,31 @@ type rawSeries struct { func seriesEquals(t *testing.T, expected []rawSeries, got []storepb.Series) { testutil.Equals(t, len(expected), len(got), "got unexpected number of series: \n %v", got) - for i, series := range got { - testutil.Equals(t, expected[i].lset, labelpb.ZLabelsToPromLabels(series.Labels)) - testutil.Equals(t, len(expected[i].chunks), len(series.Chunks), "unexpected number of chunks for series %v", series.Labels) + ret := make([]rawSeries, len(got)) + for i, s := range got { + r := rawSeries{ + lset: labelpb.ZLabelsToPromLabels(s.Labels), + } + for _, chk := range s.Chunks { + var samples []sample - for k, chk := range series.Chunks { c, err := chunkenc.FromData(chunkenc.EncXOR, chk.Raw.Data) testutil.Ok(t, err) - j := 0 iter := c.Iterator(nil) for iter.Next() != chunkenc.ValNone { - testutil.Assert(t, j < len(expected[i].chunks[k]), "more samples than expected for %v chunk %d", series.Labels, k) - tv, v := iter.At() - testutil.Equals(t, expected[i].chunks[k][j], sample{tv, v}) - j++ + samples = append(samples, sample{tv, v}) } testutil.Ok(t, iter.Err()) - testutil.Equals(t, len(expected[i].chunks[k]), j) + + r.chunks = append(r.chunks, samples) } + ret[i] = r + } + + for i := range ret { + testutil.Equals(t, expected[i], ret[i]) } } @@ -1508,7 +1575,7 @@ func TestStoreMatches(t *testing.T) { expectedReason string }{ { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "b", "1"), }, @@ -1517,7 +1584,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "does not have data within this time period: [0,-1]. Store time ranges: [0,0]", }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "b", "1"), }, @@ -1525,33 +1592,33 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{minTime: 100, maxTime: 200}, + s: &storetestutil.TestClient{MinTime: 100, MaxTime: 200}, mint: 201, maxt: 300, expectedMatch: false, expectedReason: "does not have data within this time period: [201,300]. Store time ranges: [100,200]", }, { - s: &testClient{minTime: 100, maxTime: 200}, + s: &storetestutil.TestClient{MinTime: 100, MaxTime: 200}, mint: 200, maxt: 300, expectedMatch: true, }, { - s: &testClient{minTime: 100, maxTime: 200}, + s: &storetestutil.TestClient{MinTime: 100, MaxTime: 200}, mint: 50, maxt: 99, expectedMatch: false, expectedReason: "does not have data within this time period: [50,99]. Store time ranges: [100,200]", }, { - s: &testClient{minTime: 100, maxTime: 200}, + s: &storetestutil.TestClient{MinTime: 100, MaxTime: 200}, mint: 50, maxt: 101, expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "a", "b"), }, @@ -1559,7 +1626,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "a", "c"), }, @@ -1568,7 +1635,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "external labels [{a=\"b\"}] does not match request label matchers: [a=\"c\"]", }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchRegexp, "a", "b|c"), }, @@ -1576,7 +1643,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchNotRegexp, "a", ""), }, @@ -1584,7 +1651,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{ + s: &storetestutil.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1597,7 +1664,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "external labels [{a=\"b\"} {a=\"c\"} {a=\"d\"}] does not match request label matchers: [a=\"e\"]", }, { - s: &testClient{labelSets: []labels.Labels{ + s: &storetestutil.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1609,7 +1676,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{ + s: &storetestutil.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1826,12 +1893,13 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { resps = append(resps, storepb.NewSeriesResponse(created[i])) } - clients[j] = &testClient{ + clients[j] = &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: resps, }, - minTime: math.MinInt64, - maxTime: math.MaxInt64, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + WithoutReplicaLabelsEnabled: true, } } @@ -1848,7 +1916,7 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { var expected []*storepb.Series lastLabels := storepb.Series{} for _, c := range clients { - m := c.(*testClient).StoreClient.(*mockedStoreAPI) + m := c.(*storetestutil.TestClient).StoreClient.(*mockedStoreAPI) // NOTE: Proxy will merge all series with same labels without any frame limit (https://github.com/thanos-io/thanos/issues/2332). for _, r := range m.RespSeries { @@ -1886,16 +1954,17 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { }, ) - // Change client to just one. + // Change client to one, containing all series. store.stores = func() []Client { - return []Client{&testClient{ + return []Client{&storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ // All responses. RespSeries: allResps, }, - labelSets: []labels.Labels{labels.FromStrings("ext1", "1")}, - minTime: math.MinInt64, - maxTime: math.MaxInt64, + ExtLset: []labels.Labels{labels.FromStrings("ext1", "1")}, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + WithoutReplicaLabelsEnabled: true, }} } @@ -1923,7 +1992,7 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { defer custom.TolerantVerifyLeak(t) clients := []Client{ - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ // Ensure more than 10 (internal respCh channel). @@ -1939,10 +2008,10 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "j"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: math.MinInt64, - maxTime: math.MaxInt64, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, }, - &testClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("b", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -1957,8 +2026,8 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("b", "j"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: math.MinInt64, - maxTime: math.MaxInt64, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, }, } @@ -1986,15 +2055,15 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { } func TestProxyStore_storeMatchMetadata(t *testing.T) { - c := testClient{} - c.isLocalStore = true + c := storetestutil.TestClient{Name: "testaddr"} + c.IsLocalStore = true ok, reason := storeMatchDebugMetadata(c, [][]*labels.Matcher{{}}) testutil.Assert(t, !ok) testutil.Equals(t, "the store is not remote, cannot match __address__", reason) // Change client to remote. - c.isLocalStore = false + c.IsLocalStore = false ok, reason = storeMatchDebugMetadata(c, [][]*labels.Matcher{{labels.MustNewMatcher(labels.MatchEqual, "__address__", "wrong")}}) testutil.Assert(t, !ok) diff --git a/pkg/store/storepb/rpc.pb.go b/pkg/store/storepb/rpc.pb.go index bf670d0f637..db64cf60c8d 100644 --- a/pkg/store/storepb/rpc.pb.go +++ b/pkg/store/storepb/rpc.pb.go @@ -32,6 +32,7 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +// Deprecated. Use `thanos.info` instead. type StoreType int32 const ( @@ -185,6 +186,7 @@ func (m *WriteRequest) XXX_DiscardUnknown() { var xxx_messageInfo_WriteRequest proto.InternalMessageInfo +// Deprecated. Use `thanos.info` instead. type InfoRequest struct { } @@ -221,6 +223,7 @@ func (m *InfoRequest) XXX_DiscardUnknown() { var xxx_messageInfo_InfoRequest proto.InternalMessageInfo +// Deprecated. Use `thanos.info` instead. type InfoResponse struct { // Deprecated. Use label_sets instead. Labels []github_com_thanos_io_thanos_pkg_store_labelpb.ZLabel `protobuf:"bytes,1,rep,name=labels,proto3,customtype=github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel" json:"labels"` @@ -288,10 +291,20 @@ type SeriesRequest struct { Range int64 `protobuf:"varint,11,opt,name=range,proto3" json:"range,omitempty"` // query_hints are the hints coming from the PromQL engine when // requesting a storage.SeriesSet for a given expression. + // As hints name suggest using those is best effort. QueryHints *QueryHints `protobuf:"bytes,12,opt,name=query_hints,json=queryHints,proto3" json:"query_hints,omitempty"` // shard_info is used by the querier to request a specific // shard of blocks instead of entire blocks. ShardInfo *ShardInfo `protobuf:"bytes,13,opt,name=shard_info,json=shardInfo,proto3" json:"shard_info,omitempty"` + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + WithoutReplicaLabels []string `protobuf:"bytes,14,rep,name=without_replica_labels,json=withoutReplicaLabels,proto3" json:"without_replica_labels,omitempty"` } func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } @@ -327,7 +340,10 @@ func (m *SeriesRequest) XXX_DiscardUnknown() { var xxx_messageInfo_SeriesRequest proto.InternalMessageInfo -// Analogous to storage.SelectHints. +// QueryHints represents hints from PromQL that might help to +// pre-aggregate or prepare series for faster use by clients. +// Analogous to storage.SelectHints plus additional info. +// As "hints" name suggests all of the items here are best effort. type QueryHints struct { // Query step size in milliseconds. StepMillis int64 `protobuf:"varint,1,opt,name=step_millis,json=stepMillis,proto3" json:"step_millis,omitempty"` @@ -830,89 +846,90 @@ func init() { func init() { proto.RegisterFile("store/storepb/rpc.proto", fileDescriptor_a938d55a388af629) } var fileDescriptor_a938d55a388af629 = []byte{ - // 1298 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0x5d, 0x6f, 0x13, 0x47, - 0x17, 0xf6, 0x7a, 0xbd, 0xfe, 0x38, 0x4e, 0xf2, 0x9a, 0xc1, 0xc0, 0xc6, 0x48, 0x8e, 0xdf, 0x7d, - 0xf5, 0x4a, 0x11, 0xa2, 0x36, 0x35, 0x15, 0x52, 0x2b, 0x6e, 0x92, 0x60, 0x48, 0x54, 0x62, 0xca, - 0x38, 0x21, 0x2d, 0x55, 0x65, 0xad, 0xed, 0xc9, 0x7a, 0xc5, 0x7a, 0x77, 0xd9, 0x99, 0x6d, 0xe2, - 0xdb, 0xf6, 0xbe, 0xaa, 0xfa, 0x13, 0xfa, 0x2b, 0xfa, 0x13, 0xb8, 0x2b, 0x57, 0x55, 0xd5, 0x0b, - 0xd4, 0xc2, 0x1f, 0xa9, 0xe6, 0x63, 0xd7, 0xde, 0x34, 0x40, 0x11, 0xdc, 0x44, 0x73, 0x9e, 0xe7, - 0xcc, 0x99, 0xf3, 0xed, 0x0d, 0x5c, 0xa1, 0x2c, 0x88, 0x48, 0x47, 0xfc, 0x0d, 0x47, 0x9d, 0x28, - 0x1c, 0xb7, 0xc3, 0x28, 0x60, 0x01, 0x2a, 0xb2, 0xa9, 0xed, 0x07, 0xb4, 0xb1, 0x9e, 0x55, 0x60, - 0xf3, 0x90, 0x50, 0xa9, 0xd2, 0xa8, 0x3b, 0x81, 0x13, 0x88, 0x63, 0x87, 0x9f, 0x14, 0xda, 0xca, - 0x5e, 0x08, 0xa3, 0x60, 0x76, 0xe6, 0x9e, 0x32, 0xe9, 0xd9, 0x23, 0xe2, 0x9d, 0xa5, 0x9c, 0x20, - 0x70, 0x3c, 0xd2, 0x11, 0xd2, 0x28, 0x3e, 0xee, 0xd8, 0xfe, 0x5c, 0x52, 0xd6, 0x7f, 0x60, 0xf5, - 0x28, 0x72, 0x19, 0xc1, 0x84, 0x86, 0x81, 0x4f, 0x89, 0xf5, 0xbd, 0x06, 0x2b, 0x0a, 0x79, 0x1a, - 0x13, 0xca, 0xd0, 0x16, 0x00, 0x73, 0x67, 0x84, 0x92, 0xc8, 0x25, 0xd4, 0xd4, 0x5a, 0xfa, 0x66, - 0xb5, 0x7b, 0x95, 0xdf, 0x9e, 0x11, 0x36, 0x25, 0x31, 0x1d, 0x8e, 0x83, 0x70, 0xde, 0x3e, 0x70, - 0x67, 0x64, 0x20, 0x54, 0xb6, 0x0b, 0xcf, 0x5e, 0x6c, 0xe4, 0xf0, 0xd2, 0x25, 0x74, 0x19, 0x8a, - 0x8c, 0xf8, 0xb6, 0xcf, 0xcc, 0x7c, 0x4b, 0xdb, 0xac, 0x60, 0x25, 0x21, 0x13, 0x4a, 0x11, 0x09, - 0x3d, 0x77, 0x6c, 0x9b, 0x7a, 0x4b, 0xdb, 0xd4, 0x71, 0x22, 0x5a, 0xab, 0x50, 0xdd, 0xf3, 0x8f, - 0x03, 0xe5, 0x83, 0xf5, 0x53, 0x1e, 0x56, 0xa4, 0x2c, 0xbd, 0x44, 0x63, 0x28, 0x8a, 0x40, 0x13, - 0x87, 0x56, 0xdb, 0x32, 0xb1, 0xed, 0xfb, 0x1c, 0xdd, 0xbe, 0xcd, 0x5d, 0xf8, 0xe3, 0xc5, 0xc6, - 0x27, 0x8e, 0xcb, 0xa6, 0xf1, 0xa8, 0x3d, 0x0e, 0x66, 0x1d, 0xa9, 0xf0, 0x91, 0x1b, 0xa8, 0x53, - 0x27, 0x7c, 0xe2, 0x74, 0x32, 0x39, 0x6b, 0x3f, 0x16, 0xb7, 0xb1, 0x32, 0x8d, 0xd6, 0xa1, 0x3c, - 0x73, 0xfd, 0x21, 0x0f, 0x44, 0x38, 0xae, 0xe3, 0xd2, 0xcc, 0xf5, 0x79, 0xa4, 0x82, 0xb2, 0x4f, - 0x25, 0xa5, 0x5c, 0x9f, 0xd9, 0xa7, 0x82, 0xea, 0x40, 0x45, 0x58, 0x3d, 0x98, 0x87, 0xc4, 0x2c, - 0xb4, 0xb4, 0xcd, 0xb5, 0xee, 0x85, 0xc4, 0xbb, 0x41, 0x42, 0xe0, 0x85, 0x0e, 0xba, 0x05, 0x20, - 0x1e, 0x1c, 0x52, 0xc2, 0xa8, 0x69, 0x88, 0x78, 0xd2, 0x1b, 0xd2, 0xa5, 0x01, 0x61, 0x2a, 0xad, - 0x15, 0x4f, 0xc9, 0xd4, 0xfa, 0xad, 0x00, 0xab, 0x32, 0xe5, 0x49, 0xa9, 0x96, 0x1d, 0xd6, 0x5e, - 0xef, 0x70, 0x3e, 0xeb, 0xf0, 0x2d, 0x4e, 0xb1, 0xf1, 0x94, 0x44, 0xd4, 0xd4, 0xc5, 0xeb, 0xf5, - 0x4c, 0x36, 0xf7, 0x25, 0xa9, 0x1c, 0x48, 0x75, 0x51, 0x17, 0x2e, 0x71, 0x93, 0x11, 0xa1, 0x81, - 0x17, 0x33, 0x37, 0xf0, 0x87, 0x27, 0xae, 0x3f, 0x09, 0x4e, 0x44, 0xd0, 0x3a, 0xbe, 0x38, 0xb3, - 0x4f, 0x71, 0xca, 0x1d, 0x09, 0x0a, 0x5d, 0x07, 0xb0, 0x1d, 0x27, 0x22, 0x8e, 0xcd, 0x88, 0x8c, - 0x75, 0xad, 0xbb, 0x92, 0xbc, 0xb6, 0xe5, 0x38, 0x11, 0x5e, 0xe2, 0xd1, 0x67, 0xb0, 0x1e, 0xda, - 0x11, 0x73, 0x6d, 0x8f, 0xbf, 0x22, 0x2a, 0x3f, 0x9c, 0xb8, 0xd4, 0x1e, 0x79, 0x64, 0x62, 0x16, - 0x5b, 0xda, 0x66, 0x19, 0x5f, 0x51, 0x0a, 0x49, 0x67, 0xdc, 0x51, 0x34, 0xfa, 0xfa, 0x9c, 0xbb, - 0x94, 0x45, 0x36, 0x23, 0xce, 0xdc, 0x2c, 0x89, 0xb2, 0x6c, 0x24, 0x0f, 0x7f, 0x91, 0xb5, 0x31, - 0x50, 0x6a, 0xff, 0x30, 0x9e, 0x10, 0x68, 0x03, 0xaa, 0xf4, 0x89, 0x1b, 0x0e, 0xc7, 0xd3, 0xd8, - 0x7f, 0x42, 0xcd, 0xb2, 0x70, 0x05, 0x38, 0xb4, 0x23, 0x10, 0x74, 0x0d, 0x8c, 0xa9, 0xeb, 0x33, - 0x6a, 0x56, 0x5a, 0x9a, 0x48, 0xa8, 0x9c, 0xc0, 0x76, 0x32, 0x81, 0xed, 0x2d, 0x7f, 0x8e, 0xa5, - 0x0a, 0x42, 0x50, 0xa0, 0x8c, 0x84, 0x26, 0x88, 0xb4, 0x89, 0x33, 0xaa, 0x83, 0x11, 0xd9, 0xbe, - 0x43, 0xcc, 0xaa, 0x00, 0xa5, 0x80, 0x6e, 0x42, 0xf5, 0x69, 0x4c, 0xa2, 0xf9, 0x50, 0xda, 0x5e, - 0x11, 0xb6, 0x51, 0x12, 0xc5, 0x43, 0x4e, 0xed, 0x72, 0x06, 0xc3, 0xd3, 0xf4, 0x8c, 0x6e, 0x00, - 0xd0, 0xa9, 0x1d, 0x4d, 0x86, 0xae, 0x7f, 0x1c, 0x98, 0xab, 0xe2, 0xce, 0xa2, 0x21, 0x39, 0x23, - 0x26, 0xab, 0x42, 0x93, 0xa3, 0xf5, 0xb3, 0x06, 0xb0, 0x30, 0x26, 0x82, 0x65, 0x24, 0x1c, 0xce, - 0x5c, 0xcf, 0x73, 0xa9, 0x6a, 0x2c, 0xe0, 0xd0, 0xbe, 0x40, 0x50, 0x0b, 0x0a, 0xc7, 0xb1, 0x3f, - 0x16, 0x7d, 0x55, 0x5d, 0x94, 0xf3, 0x6e, 0xec, 0x8f, 0xb1, 0x60, 0xd0, 0x75, 0x28, 0x3b, 0x51, - 0x10, 0x87, 0xae, 0xef, 0x88, 0xee, 0xa8, 0x76, 0x6b, 0x89, 0xd6, 0x3d, 0x85, 0xe3, 0x54, 0x03, - 0xfd, 0x2f, 0x09, 0xde, 0x10, 0xaa, 0xe9, 0x6c, 0x63, 0x0e, 0xaa, 0x5c, 0x58, 0x27, 0x50, 0x49, - 0x9d, 0x17, 0x2e, 0xaa, 0x18, 0x27, 0xe4, 0x34, 0x75, 0x51, 0xf2, 0x13, 0x72, 0x8a, 0xfe, 0x0b, - 0x2b, 0x2c, 0x60, 0xb6, 0x37, 0x14, 0x18, 0x55, 0x23, 0x50, 0x15, 0x98, 0x30, 0x43, 0xd1, 0x1a, - 0xe4, 0x47, 0x73, 0x31, 0xcc, 0x65, 0x9c, 0x1f, 0xcd, 0xf9, 0xd2, 0x52, 0x2b, 0xa6, 0xd0, 0xd2, - 0xf9, 0xd2, 0x92, 0x92, 0xd5, 0x80, 0x02, 0x8f, 0x8c, 0x97, 0xcd, 0xb7, 0xd5, 0xa0, 0x55, 0xb0, - 0x38, 0x5b, 0x5d, 0x28, 0x27, 0xf1, 0x28, 0x7b, 0xda, 0x39, 0xf6, 0xf4, 0x8c, 0xbd, 0x0d, 0x30, - 0x44, 0x60, 0x5c, 0x21, 0x93, 0x62, 0x25, 0x59, 0x3f, 0x68, 0xb0, 0x96, 0xcc, 0xb9, 0x5a, 0x7f, - 0x9b, 0x50, 0x4c, 0xf7, 0x31, 0x4f, 0xd1, 0x5a, 0x5a, 0x4f, 0x81, 0xee, 0xe6, 0xb0, 0xe2, 0x51, - 0x03, 0x4a, 0x27, 0x76, 0xe4, 0xf3, 0xc4, 0x8b, 0xdd, 0xbb, 0x9b, 0xc3, 0x09, 0x80, 0xae, 0x27, - 0x4d, 0xaa, 0xbf, 0xbe, 0x49, 0x77, 0x73, 0xaa, 0x4d, 0xb7, 0xcb, 0x50, 0x8c, 0x08, 0x8d, 0x3d, - 0x66, 0xfd, 0x92, 0x87, 0x0b, 0x62, 0x33, 0xf4, 0xed, 0xd9, 0x62, 0xf9, 0xbc, 0x71, 0x58, 0xb5, - 0xf7, 0x18, 0xd6, 0xfc, 0x7b, 0x0e, 0x6b, 0x1d, 0x0c, 0xca, 0xec, 0x88, 0xa9, 0x45, 0x2d, 0x05, - 0x54, 0x03, 0x9d, 0xf8, 0x13, 0xb5, 0xab, 0xf8, 0x71, 0x31, 0xb3, 0xc6, 0xdb, 0x67, 0x76, 0x79, - 0x67, 0x16, 0xff, 0xfd, 0xce, 0xb4, 0x22, 0x40, 0xcb, 0x99, 0x53, 0xe5, 0xac, 0x83, 0xc1, 0xdb, - 0x47, 0xfe, 0x98, 0x55, 0xb0, 0x14, 0x50, 0x03, 0xca, 0xaa, 0x52, 0xbc, 0x5f, 0x39, 0x91, 0xca, - 0x0b, 0x5f, 0xf5, 0xb7, 0xfa, 0x6a, 0xfd, 0x9a, 0x57, 0x8f, 0x3e, 0xb2, 0xbd, 0x78, 0x51, 0xaf, - 0x3a, 0x18, 0xa2, 0x03, 0x55, 0x03, 0x4b, 0xe1, 0xcd, 0x55, 0xcc, 0xbf, 0x47, 0x15, 0xf5, 0x0f, - 0x55, 0xc5, 0xc2, 0x39, 0x55, 0x34, 0xce, 0xa9, 0x62, 0xf1, 0xdd, 0xaa, 0x58, 0x7a, 0x87, 0x2a, - 0xc6, 0x70, 0x31, 0x93, 0x50, 0x55, 0xc6, 0xcb, 0x50, 0xfc, 0x56, 0x20, 0xaa, 0x8e, 0x4a, 0xfa, - 0x50, 0x85, 0xbc, 0xf6, 0x0d, 0x54, 0xd2, 0x0f, 0x08, 0x54, 0x85, 0xd2, 0x61, 0xff, 0xf3, 0xfe, - 0x83, 0xa3, 0x7e, 0x2d, 0x87, 0x2a, 0x60, 0x3c, 0x3c, 0xec, 0xe1, 0xaf, 0x6a, 0x1a, 0x2a, 0x43, - 0x01, 0x1f, 0xde, 0xef, 0xd5, 0xf2, 0x5c, 0x63, 0xb0, 0x77, 0xa7, 0xb7, 0xb3, 0x85, 0x6b, 0x3a, - 0xd7, 0x18, 0x1c, 0x3c, 0xc0, 0xbd, 0x5a, 0x81, 0xe3, 0xb8, 0xb7, 0xd3, 0xdb, 0x7b, 0xd4, 0xab, - 0x19, 0x1c, 0xbf, 0xd3, 0xdb, 0x3e, 0xbc, 0x57, 0x2b, 0x5e, 0xdb, 0x86, 0x02, 0xff, 0x05, 0x46, - 0x25, 0xd0, 0xf1, 0xd6, 0x91, 0xb4, 0xba, 0xf3, 0xe0, 0xb0, 0x7f, 0x50, 0xd3, 0x38, 0x36, 0x38, - 0xdc, 0xaf, 0xe5, 0xf9, 0x61, 0x7f, 0xaf, 0x5f, 0xd3, 0xc5, 0x61, 0xeb, 0x4b, 0x69, 0x4e, 0x68, - 0xf5, 0x70, 0xcd, 0xe8, 0x7e, 0x97, 0x07, 0x43, 0xf8, 0x88, 0x3e, 0x86, 0x82, 0x58, 0xcd, 0x17, - 0x93, 0x8c, 0x2e, 0x7d, 0xcf, 0x35, 0xea, 0x59, 0x50, 0xe5, 0xef, 0x53, 0x28, 0xca, 0xfd, 0x85, - 0x2e, 0x65, 0xf7, 0x59, 0x72, 0xed, 0xf2, 0x59, 0x58, 0x5e, 0xbc, 0xa1, 0xa1, 0x1d, 0x80, 0xc5, - 0x5c, 0xa1, 0xf5, 0x4c, 0x15, 0x97, 0xb7, 0x54, 0xa3, 0x71, 0x1e, 0xa5, 0xde, 0xbf, 0x0b, 0xd5, - 0xa5, 0xb2, 0xa2, 0xac, 0x6a, 0x66, 0x78, 0x1a, 0x57, 0xcf, 0xe5, 0xa4, 0x9d, 0x6e, 0x1f, 0xd6, - 0xc4, 0x17, 0x34, 0x9f, 0x0a, 0x99, 0x8c, 0xdb, 0x50, 0xc5, 0x64, 0x16, 0x30, 0x22, 0x70, 0x94, - 0x86, 0xbf, 0xfc, 0xa1, 0xdd, 0xb8, 0x74, 0x06, 0x55, 0x1f, 0xe4, 0xb9, 0xed, 0xff, 0x3f, 0xfb, - 0xab, 0x99, 0x7b, 0xf6, 0xb2, 0xa9, 0x3d, 0x7f, 0xd9, 0xd4, 0xfe, 0x7c, 0xd9, 0xd4, 0x7e, 0x7c, - 0xd5, 0xcc, 0x3d, 0x7f, 0xd5, 0xcc, 0xfd, 0xfe, 0xaa, 0x99, 0x7b, 0x5c, 0x52, 0xff, 0x13, 0x8c, - 0x8a, 0xa2, 0x67, 0x6e, 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x02, 0x42, 0x0e, 0xd0, 0x7d, 0x0c, - 0x00, 0x00, + // 1323 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0xdd, 0x6e, 0x13, 0xc7, + 0x17, 0xf7, 0x7a, 0xbd, 0xfe, 0x38, 0x4e, 0xf2, 0x37, 0x83, 0x09, 0x1b, 0x23, 0x39, 0xfe, 0xbb, + 0xaa, 0x64, 0x21, 0x6a, 0x53, 0x83, 0x90, 0x5a, 0x71, 0x93, 0x04, 0x43, 0xa2, 0x12, 0x53, 0xc6, + 0x09, 0x69, 0xa9, 0x2a, 0x6b, 0x6d, 0x4f, 0xd6, 0x2b, 0xec, 0xdd, 0x65, 0x67, 0xb6, 0x89, 0x6f, + 0xdb, 0xdb, 0xaa, 0xaa, 0xfa, 0x08, 0x7d, 0x8a, 0x3e, 0x02, 0x77, 0xe5, 0xb2, 0xea, 0x05, 0x6a, + 0xe1, 0x45, 0xaa, 0x39, 0x3b, 0x6b, 0x7b, 0xd3, 0x00, 0x45, 0x70, 0x13, 0xcd, 0xf9, 0xfd, 0xce, + 0x9c, 0x39, 0xdf, 0xde, 0xc0, 0x65, 0x2e, 0xbc, 0x80, 0xb5, 0xf0, 0xaf, 0x3f, 0x68, 0x05, 0xfe, + 0xb0, 0xe9, 0x07, 0x9e, 0xf0, 0x48, 0x56, 0x8c, 0x2d, 0xd7, 0xe3, 0x95, 0x8d, 0xa4, 0x82, 0x98, + 0xf9, 0x8c, 0x47, 0x2a, 0x95, 0xb2, 0xed, 0xd9, 0x1e, 0x1e, 0x5b, 0xf2, 0xa4, 0xd0, 0x5a, 0xf2, + 0x82, 0x1f, 0x78, 0xd3, 0x33, 0xf7, 0x94, 0xc9, 0x89, 0x35, 0x60, 0x93, 0xb3, 0x94, 0xed, 0x79, + 0xf6, 0x84, 0xb5, 0x50, 0x1a, 0x84, 0xc7, 0x2d, 0xcb, 0x9d, 0x45, 0x54, 0xfd, 0x7f, 0xb0, 0x7a, + 0x14, 0x38, 0x82, 0x51, 0xc6, 0x7d, 0xcf, 0xe5, 0xac, 0xfe, 0x83, 0x06, 0x2b, 0x0a, 0x79, 0x1a, + 0x32, 0x2e, 0xc8, 0x16, 0x80, 0x70, 0xa6, 0x8c, 0xb3, 0xc0, 0x61, 0xdc, 0xd4, 0x6a, 0x7a, 0xa3, + 0xd8, 0xbe, 0x22, 0x6f, 0x4f, 0x99, 0x18, 0xb3, 0x90, 0xf7, 0x87, 0x9e, 0x3f, 0x6b, 0x1e, 0x38, + 0x53, 0xd6, 0x43, 0x95, 0xed, 0xcc, 0xb3, 0x17, 0x9b, 0x29, 0xba, 0x74, 0x89, 0xac, 0x43, 0x56, + 0x30, 0xd7, 0x72, 0x85, 0x99, 0xae, 0x69, 0x8d, 0x02, 0x55, 0x12, 0x31, 0x21, 0x17, 0x30, 0x7f, + 0xe2, 0x0c, 0x2d, 0x53, 0xaf, 0x69, 0x0d, 0x9d, 0xc6, 0x62, 0x7d, 0x15, 0x8a, 0x7b, 0xee, 0xb1, + 0xa7, 0x7c, 0xa8, 0xff, 0x92, 0x86, 0x95, 0x48, 0x8e, 0xbc, 0x24, 0x43, 0xc8, 0x62, 0xa0, 0xb1, + 0x43, 0xab, 0xcd, 0x28, 0xb1, 0xcd, 0xfb, 0x12, 0xdd, 0xbe, 0x2d, 0x5d, 0xf8, 0xf3, 0xc5, 0xe6, + 0x4d, 0xdb, 0x11, 0xe3, 0x70, 0xd0, 0x1c, 0x7a, 0xd3, 0x56, 0xa4, 0xf0, 0x89, 0xe3, 0xa9, 0x53, + 0xcb, 0x7f, 0x62, 0xb7, 0x12, 0x39, 0x6b, 0x3e, 0xc6, 0xdb, 0x54, 0x99, 0x26, 0x1b, 0x90, 0x9f, + 0x3a, 0x6e, 0x5f, 0x06, 0x82, 0x8e, 0xeb, 0x34, 0x37, 0x75, 0x5c, 0x19, 0x29, 0x52, 0xd6, 0x69, + 0x44, 0x29, 0xd7, 0xa7, 0xd6, 0x29, 0x52, 0x2d, 0x28, 0xa0, 0xd5, 0x83, 0x99, 0xcf, 0xcc, 0x4c, + 0x4d, 0x6b, 0xac, 0xb5, 0x2f, 0xc4, 0xde, 0xf5, 0x62, 0x82, 0x2e, 0x74, 0xc8, 0x2d, 0x00, 0x7c, + 0xb0, 0xcf, 0x99, 0xe0, 0xa6, 0x81, 0xf1, 0xcc, 0x6f, 0x44, 0x2e, 0xf5, 0x98, 0x50, 0x69, 0x2d, + 0x4c, 0x94, 0xcc, 0xeb, 0x3f, 0x1a, 0xb0, 0x1a, 0xa5, 0x3c, 0x2e, 0xd5, 0xb2, 0xc3, 0xda, 0xeb, + 0x1d, 0x4e, 0x27, 0x1d, 0xbe, 0x25, 0x29, 0x31, 0x1c, 0xb3, 0x80, 0x9b, 0x3a, 0xbe, 0x5e, 0x4e, + 0x64, 0x73, 0x3f, 0x22, 0x95, 0x03, 0x73, 0x5d, 0xd2, 0x86, 0x4b, 0xd2, 0x64, 0xc0, 0xb8, 0x37, + 0x09, 0x85, 0xe3, 0xb9, 0xfd, 0x13, 0xc7, 0x1d, 0x79, 0x27, 0x18, 0xb4, 0x4e, 0x2f, 0x4e, 0xad, + 0x53, 0x3a, 0xe7, 0x8e, 0x90, 0x22, 0xd7, 0x00, 0x2c, 0xdb, 0x0e, 0x98, 0x6d, 0x09, 0x16, 0xc5, + 0xba, 0xd6, 0x5e, 0x89, 0x5f, 0xdb, 0xb2, 0xed, 0x80, 0x2e, 0xf1, 0xe4, 0x73, 0xd8, 0xf0, 0xad, + 0x40, 0x38, 0xd6, 0x44, 0xbe, 0x82, 0x95, 0xef, 0x8f, 0x1c, 0x6e, 0x0d, 0x26, 0x6c, 0x64, 0x66, + 0x6b, 0x5a, 0x23, 0x4f, 0x2f, 0x2b, 0x85, 0xb8, 0x33, 0xee, 0x28, 0x9a, 0x7c, 0x73, 0xce, 0x5d, + 0x2e, 0x02, 0x4b, 0x30, 0x7b, 0x66, 0xe6, 0xb0, 0x2c, 0x9b, 0xf1, 0xc3, 0x5f, 0x26, 0x6d, 0xf4, + 0x94, 0xda, 0xbf, 0x8c, 0xc7, 0x04, 0xd9, 0x84, 0x22, 0x7f, 0xe2, 0xf8, 0xfd, 0xe1, 0x38, 0x74, + 0x9f, 0x70, 0x33, 0x8f, 0xae, 0x80, 0x84, 0x76, 0x10, 0x21, 0x57, 0xc1, 0x18, 0x3b, 0xae, 0xe0, + 0x66, 0xa1, 0xa6, 0x61, 0x42, 0xa3, 0x09, 0x6c, 0xc6, 0x13, 0xd8, 0xdc, 0x72, 0x67, 0x34, 0x52, + 0x21, 0x04, 0x32, 0x5c, 0x30, 0xdf, 0x04, 0x4c, 0x1b, 0x9e, 0x49, 0x19, 0x8c, 0xc0, 0x72, 0x6d, + 0x66, 0x16, 0x11, 0x8c, 0x04, 0x72, 0x03, 0x8a, 0x4f, 0x43, 0x16, 0xcc, 0xfa, 0x91, 0xed, 0x15, + 0xb4, 0x4d, 0xe2, 0x28, 0x1e, 0x4a, 0x6a, 0x57, 0x32, 0x14, 0x9e, 0xce, 0xcf, 0xe4, 0x3a, 0x00, + 0x1f, 0x5b, 0xc1, 0xa8, 0xef, 0xb8, 0xc7, 0x9e, 0xb9, 0x8a, 0x77, 0x16, 0x0d, 0x29, 0x19, 0x9c, + 0xac, 0x02, 0x8f, 0x8f, 0xe4, 0x26, 0xac, 0x9f, 0x38, 0x62, 0xec, 0x85, 0xa2, 0xaf, 0xe6, 0xb1, + 0xaf, 0x86, 0x6d, 0xad, 0xa6, 0x37, 0x0a, 0xb4, 0xac, 0x58, 0x1a, 0x91, 0xd8, 0x24, 0xbc, 0xfe, + 0xab, 0x06, 0xb0, 0x70, 0x01, 0x53, 0x24, 0x98, 0xdf, 0x9f, 0x3a, 0x93, 0x89, 0xc3, 0x55, 0x3b, + 0x82, 0x84, 0xf6, 0x11, 0x21, 0x35, 0xc8, 0x1c, 0x87, 0xee, 0x10, 0xbb, 0xb1, 0xb8, 0x68, 0x82, + 0xbb, 0xa1, 0x3b, 0xa4, 0xc8, 0x90, 0x6b, 0x90, 0xb7, 0x03, 0x2f, 0xf4, 0x1d, 0xd7, 0xc6, 0x9e, + 0x2a, 0xb6, 0x4b, 0xb1, 0xd6, 0x3d, 0x85, 0xd3, 0xb9, 0x06, 0xf9, 0x28, 0x4e, 0x99, 0x81, 0xaa, + 0xf3, 0x8d, 0x40, 0x25, 0xa8, 0x32, 0x58, 0x3f, 0x81, 0xc2, 0x3c, 0x64, 0x74, 0x51, 0x65, 0x66, + 0xc4, 0x4e, 0xe7, 0x2e, 0x46, 0xfc, 0x88, 0x9d, 0x92, 0xff, 0xc3, 0x8a, 0xf0, 0x84, 0x35, 0xe9, + 0x23, 0xc6, 0xd5, 0xe0, 0x14, 0x11, 0x43, 0x33, 0x9c, 0xac, 0x41, 0x7a, 0x30, 0xc3, 0x15, 0x90, + 0xa7, 0xe9, 0xc1, 0x4c, 0xae, 0x3a, 0x95, 0xab, 0x0c, 0xe6, 0x4a, 0x49, 0xf5, 0x0a, 0x64, 0x64, + 0x64, 0xb2, 0xd8, 0xae, 0xa5, 0xc6, 0xb3, 0x40, 0xf1, 0x5c, 0x6f, 0x43, 0x3e, 0x8e, 0x47, 0xd9, + 0xd3, 0xce, 0xb1, 0xa7, 0x27, 0xec, 0x6d, 0x82, 0x81, 0x81, 0x49, 0x85, 0x44, 0x8a, 0x95, 0x54, + 0xff, 0x49, 0x83, 0xb5, 0x78, 0x3b, 0xa8, 0xa5, 0xd9, 0x80, 0xec, 0x7c, 0x8b, 0xcb, 0x14, 0xad, + 0xcd, 0xbb, 0x00, 0xd1, 0xdd, 0x14, 0x55, 0x3c, 0xa9, 0x40, 0xee, 0xc4, 0x0a, 0x5c, 0x99, 0x78, + 0xdc, 0xd8, 0xbb, 0x29, 0x1a, 0x03, 0xe4, 0x5a, 0xdc, 0xda, 0xfa, 0xeb, 0x5b, 0x7b, 0x37, 0xa5, + 0x9a, 0x7b, 0x3b, 0x0f, 0xd9, 0x80, 0xf1, 0x70, 0x22, 0xea, 0xbf, 0xa5, 0xe1, 0x02, 0xb6, 0x4a, + 0xd7, 0x9a, 0x2e, 0x56, 0xd6, 0x1b, 0x47, 0x5c, 0x7b, 0x8f, 0x11, 0x4f, 0xbf, 0xe7, 0x88, 0x97, + 0xc1, 0xe0, 0xc2, 0x0a, 0x84, 0x5a, 0xef, 0x91, 0x40, 0x4a, 0xa0, 0x33, 0x77, 0xa4, 0x36, 0x9c, + 0x3c, 0x2e, 0x26, 0xdd, 0x78, 0xfb, 0xa4, 0x2f, 0x6f, 0xda, 0xec, 0x7f, 0xdf, 0xb4, 0xf5, 0x00, + 0xc8, 0x72, 0xe6, 0x54, 0x39, 0xcb, 0x60, 0xc8, 0xf6, 0x89, 0x7e, 0x02, 0x0b, 0x34, 0x12, 0x48, + 0x05, 0xf2, 0xaa, 0x52, 0xb2, 0x5f, 0x25, 0x31, 0x97, 0x17, 0xbe, 0xea, 0x6f, 0xf5, 0xb5, 0xfe, + 0x7b, 0x5a, 0x3d, 0xfa, 0xc8, 0x9a, 0x84, 0x8b, 0x7a, 0x95, 0xc1, 0xc0, 0x0e, 0x54, 0x0d, 0x1c, + 0x09, 0x6f, 0xae, 0x62, 0xfa, 0x3d, 0xaa, 0xa8, 0x7f, 0xa8, 0x2a, 0x66, 0xce, 0xa9, 0xa2, 0x71, + 0x4e, 0x15, 0xb3, 0xef, 0x56, 0xc5, 0xdc, 0x3b, 0x54, 0x31, 0x84, 0x8b, 0x89, 0x84, 0xaa, 0x32, + 0xae, 0x43, 0xf6, 0x3b, 0x44, 0x54, 0x1d, 0x95, 0xf4, 0xa1, 0x0a, 0x79, 0xf5, 0x5b, 0x28, 0xcc, + 0x3f, 0x3b, 0x48, 0x11, 0x72, 0x87, 0xdd, 0x2f, 0xba, 0x0f, 0x8e, 0xba, 0xa5, 0x14, 0x29, 0x80, + 0xf1, 0xf0, 0xb0, 0x43, 0xbf, 0x2e, 0x69, 0x24, 0x0f, 0x19, 0x7a, 0x78, 0xbf, 0x53, 0x4a, 0x4b, + 0x8d, 0xde, 0xde, 0x9d, 0xce, 0xce, 0x16, 0x2d, 0xe9, 0x52, 0xa3, 0x77, 0xf0, 0x80, 0x76, 0x4a, + 0x19, 0x89, 0xd3, 0xce, 0x4e, 0x67, 0xef, 0x51, 0xa7, 0x64, 0x48, 0xfc, 0x4e, 0x67, 0xfb, 0xf0, + 0x5e, 0x29, 0x7b, 0x75, 0x1b, 0x32, 0xf2, 0x77, 0x9b, 0xe4, 0x40, 0xa7, 0x5b, 0x47, 0x91, 0xd5, + 0x9d, 0x07, 0x87, 0xdd, 0x83, 0x92, 0x26, 0xb1, 0xde, 0xe1, 0x7e, 0x29, 0x2d, 0x0f, 0xfb, 0x7b, + 0xdd, 0x92, 0x8e, 0x87, 0xad, 0xaf, 0x22, 0x73, 0xa8, 0xd5, 0xa1, 0x25, 0xa3, 0xfd, 0x7d, 0x1a, + 0x0c, 0xf4, 0x91, 0x7c, 0x0a, 0x19, 0x5c, 0xcd, 0x17, 0xe3, 0x8c, 0x2e, 0x7d, 0x05, 0x56, 0xca, + 0x49, 0x50, 0xe5, 0xef, 0x33, 0xc8, 0x46, 0xfb, 0x8b, 0x5c, 0x4a, 0xee, 0xb3, 0xf8, 0xda, 0xfa, + 0x59, 0x38, 0xba, 0x78, 0x5d, 0x23, 0x3b, 0x00, 0x8b, 0xb9, 0x22, 0x1b, 0x89, 0x2a, 0x2e, 0x6f, + 0xa9, 0x4a, 0xe5, 0x3c, 0x4a, 0xbd, 0x7f, 0x17, 0x8a, 0x4b, 0x65, 0x25, 0x49, 0xd5, 0xc4, 0xf0, + 0x54, 0xae, 0x9c, 0xcb, 0x45, 0x76, 0xda, 0x5d, 0x58, 0xc3, 0xef, 0x6e, 0x39, 0x15, 0x51, 0x32, + 0x6e, 0x43, 0x91, 0xb2, 0xa9, 0x27, 0x18, 0xe2, 0x64, 0x1e, 0xfe, 0xf2, 0xe7, 0x79, 0xe5, 0xd2, + 0x19, 0x54, 0x7d, 0xc6, 0xa7, 0xb6, 0x3f, 0x7e, 0xf6, 0x77, 0x35, 0xf5, 0xec, 0x65, 0x55, 0x7b, + 0xfe, 0xb2, 0xaa, 0xfd, 0xf5, 0xb2, 0xaa, 0xfd, 0xfc, 0xaa, 0x9a, 0x7a, 0xfe, 0xaa, 0x9a, 0xfa, + 0xe3, 0x55, 0x35, 0xf5, 0x38, 0xa7, 0xfe, 0x93, 0x18, 0x64, 0xb1, 0x67, 0x6e, 0xfc, 0x13, 0x00, + 0x00, 0xff, 0xff, 0x73, 0x1f, 0x05, 0x4d, 0xb3, 0x0c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -929,6 +946,7 @@ const _ = grpc.SupportPackageIsVersion4 type StoreClient interface { /// Info returns meta information about a store e.g labels that makes that store unique as well as time range that is /// available. + /// Deprecated. Use `thanos.info` instead. Info(ctx context.Context, in *InfoRequest, opts ...grpc.CallOption) (*InfoResponse, error) /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. /// @@ -1017,6 +1035,7 @@ func (c *storeClient) LabelValues(ctx context.Context, in *LabelValuesRequest, o type StoreServer interface { /// Info returns meta information about a store e.g labels that makes that store unique as well as time range that is /// available. + /// Deprecated. Use `thanos.info` instead. Info(context.Context, *InfoRequest) (*InfoResponse, error) /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. /// @@ -1412,6 +1431,15 @@ func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.WithoutReplicaLabels) > 0 { + for iNdEx := len(m.WithoutReplicaLabels) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.WithoutReplicaLabels[iNdEx]) + copy(dAtA[i:], m.WithoutReplicaLabels[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.WithoutReplicaLabels[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } if m.ShardInfo != nil { { size, err := m.ShardInfo.MarshalToSizedBuffer(dAtA[:i]) @@ -2234,6 +2262,12 @@ func (m *SeriesRequest) Size() (n int) { l = m.ShardInfo.Size() n += 1 + l + sovRpc(uint64(l)) } + if len(m.WithoutReplicaLabels) > 0 { + for _, s := range m.WithoutReplicaLabels { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } return n } @@ -3300,6 +3334,38 @@ func (m *SeriesRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithoutReplicaLabels", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithoutReplicaLabels = append(m.WithoutReplicaLabels, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) diff --git a/pkg/store/storepb/rpc.proto b/pkg/store/storepb/rpc.proto index 72afaba8ed2..9e5ebaa693d 100644 --- a/pkg/store/storepb/rpc.proto +++ b/pkg/store/storepb/rpc.proto @@ -27,6 +27,7 @@ option (gogoproto.goproto_sizecache_all) = false; service Store { /// Info returns meta information about a store e.g labels that makes that store unique as well as time range that is /// available. + /// Deprecated. Use `thanos.info` instead. rpc Info(InfoRequest) returns (InfoResponse); /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. @@ -62,8 +63,10 @@ message WriteRequest { int64 replica = 3; } +// Deprecated. Use `thanos.info` instead. message InfoRequest {} +// Deprecated. Use `thanos.info` instead. enum StoreType { UNKNOWN = 0; QUERY = 1; @@ -75,6 +78,7 @@ enum StoreType { DEBUG = 6; } +// Deprecated. Use `thanos.info` instead. message InfoResponse { // Deprecated. Use label_sets instead. repeated Label labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; @@ -117,15 +121,28 @@ message SeriesRequest { // query_hints are the hints coming from the PromQL engine when // requesting a storage.SeriesSet for a given expression. + // As hints name suggest using those is best effort. QueryHints query_hints = 12; // shard_info is used by the querier to request a specific // shard of blocks instead of entire blocks. ShardInfo shard_info = 13; -} + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + repeated string without_replica_labels = 14; +} -// Analogous to storage.SelectHints. +// QueryHints represents hints from PromQL that might help to +// pre-aggregate or prepare series for faster use by clients. +// Analogous to storage.SelectHints plus additional info. +// As "hints" name suggests all of the items here are best effort. message QueryHints { // Query step size in milliseconds. int64 step_millis = 1; diff --git a/pkg/store/storepb/testutil/client.go b/pkg/store/storepb/testutil/client.go new file mode 100644 index 00000000000..b6916005a38 --- /dev/null +++ b/pkg/store/storepb/testutil/client.go @@ -0,0 +1,28 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storetestutil + +import ( + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/thanos/pkg/store/storepb" +) + +type TestClient struct { + storepb.StoreClient + + Name string + + ExtLset []labels.Labels + MinTime, MaxTime int64 + Shardable bool + WithoutReplicaLabelsEnabled bool + IsLocalStore bool +} + +func (c TestClient) LabelSets() []labels.Labels { return c.ExtLset } +func (c TestClient) TimeRange() (mint, maxt int64) { return c.MinTime, c.MaxTime } +func (c TestClient) SupportsSharding() bool { return c.Shardable } +func (c TestClient) SupportsWithoutReplicaLabels() bool { return c.WithoutReplicaLabelsEnabled } +func (c TestClient) String() string { return c.Name } +func (c TestClient) Addr() (string, bool) { return c.Name, c.IsLocalStore } diff --git a/test/e2e/query_frontend_test.go b/test/e2e/query_frontend_test.go index 1d59420748d..c3f0b8756c2 100644 --- a/test/e2e/query_frontend_test.go +++ b/test/e2e/query_frontend_test.go @@ -6,6 +6,7 @@ package e2e_test import ( "context" "reflect" + "sort" "testing" "time" @@ -713,6 +714,13 @@ func TestInstantQueryShardingWithRandomData(t *testing.T) { {{Name: labels.MetricName, Value: "http_requests_total"}, {Name: "pod", Value: "6"}, {Name: "handler", Value: "/metrics"}}, } + // Ensure labels are ordered. + for _, ts := range timeSeries { + sort.Slice(ts, func(i, j int) bool { + return ts[i].Name < ts[j].Name + }) + } + startTime := now.Time().Add(-1 * time.Hour) endTime := now.Time().Add(1 * time.Hour) _, err = e2eutil.CreateBlock(ctx, prom.Dir(), timeSeries, 20, timestamp.FromTime(startTime), timestamp.FromTime(endTime), nil, 0, metadata.NoneFunc)