diff --git a/CHANGELOG.md b/CHANGELOG.md index c8df2c9339..e9cf38a89c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -31,6 +31,7 @@ We use *breaking* word for marking changes that are not backward compatible (rel - [2513](https://github.com/thanos-io/thanos/pull/2513) Tools: Moved `thanos bucket` commands to `thanos tools bucket`, also moved `thanos check rules` to `thanos tools rules-check`. `thanos tools rules-check` also takes rules by `--rules` repeated flag not argument anymore. +- [2603](https://github.com/thanos-io/thanos/pull/2603) Store/Querier: Significantly optimize cases where StoreAPIs or blocks returns exact overlapping chunks (e.g Store GW and sidecar or brute force Store Gateway HA). ## [v0.12.2](https://github.com/thanos-io/thanos/releases/tag/v0.12.2) - 2020.04.30 diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index fa3ff3593d..426849eed5 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -981,9 +981,8 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie tracing.DoInSpan(ctx, "bucket_store_merge_all", func(ctx context.Context) { begin := time.Now() - // Merge series set into an union of all block sets. This exposes all blocks are single seriesSet. - // Chunks of returned series might be out of order w.r.t to their time range. - // This must be accounted for later by clients. + // NOTE: We "carefully" assume series and chunks are sorted within each SeriesSet. This should be guaranteed by + // blockSeries method. In worst case deduplication logic won't deduplicate correctly, which will be accounted later. set := storepb.MergeSeriesSets(res...) for set.Next() { var series storepb.Series diff --git a/pkg/store/proxy.go b/pkg/store/proxy.go index 36da2e62fd..4fc34eb850 100644 --- a/pkg/store/proxy.go +++ b/pkg/store/proxy.go @@ -473,6 +473,7 @@ func (s *streamSeriesSet) At() ([]storepb.Label, []storepb.AggrChunk) { } return s.currSeries.Labels, s.currSeries.Chunks } + func (s *streamSeriesSet) Err() error { s.errMtx.Lock() defer s.errMtx.Unlock() diff --git a/pkg/store/proxy_test.go b/pkg/store/proxy_test.go index f4e8849904..3235a3220e 100644 --- a/pkg/store/proxy_test.go +++ b/pkg/store/proxy_test.go @@ -294,15 +294,11 @@ func TestProxyStore_Series(t *testing.T) { expectedSeries: []rawSeries{ { lset: []storepb.Label{{Name: "a", Value: "a"}}, - chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{4, 3}}}, - }, - { - lset: []storepb.Label{{Name: "a", Value: "a"}}, - chunks: [][]sample{{{5, 4}}}, + chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{4, 3}}, {{5, 4}}}, }, { lset: []storepb.Label{{Name: "a", Value: "b"}}, - chunks: [][]sample{{{2, 2}, {3, 3}, {4, 4}}, {{1, 1}, {2, 2}, {3, 3}}}, // No sort merge. + chunks: [][]sample{{{1, 1}, {2, 2}, {3, 3}}, {{2, 2}, {3, 3}, {4, 4}}}, }, { lset: []storepb.Label{{Name: "a", Value: "c"}}, @@ -343,7 +339,7 @@ func TestProxyStore_Series(t *testing.T) { expectedSeries: []rawSeries{ { lset: []storepb.Label{{Name: "a", Value: "b"}}, - chunks: [][]sample{{{1, 1}, {2, 2}, {3, 3}}, {{1, 11}, {2, 22}, {3, 33}}}, + chunks: [][]sample{{{1, 11}, {2, 22}, {3, 33}}, {{1, 1}, {2, 2}, {3, 3}}}, }, }, }, @@ -1220,7 +1216,7 @@ type rawSeries struct { } func seriesEquals(t *testing.T, expected []rawSeries, got []storepb.Series) { - testutil.Equals(t, len(expected), len(got), "got: %v", got) + 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, series.Labels) diff --git a/pkg/store/storepb/custom.go b/pkg/store/storepb/custom.go index 781c6d761f..dfc1a4f8ec 100644 --- a/pkg/store/storepb/custom.go +++ b/pkg/store/storepb/custom.go @@ -4,6 +4,7 @@ package storepb import ( + "bytes" "strings" "unsafe" @@ -45,6 +46,7 @@ func NewHintsSeriesResponse(hints *types.Any) *SeriesResponse { } // CompareLabels compares two sets of labels. +// After lexicographical order, the set with fewer labels comes first. func CompareLabels(a, b []Label) int { l := len(a) if len(b) < l { @@ -58,7 +60,7 @@ func CompareLabels(a, b []Label) int { return d } } - // If all labels so far were in common, the set with fewer labels comes first. + return len(a) - len(b) } @@ -73,13 +75,25 @@ func EmptySeriesSet() SeriesSet { return emptySeriesSet{} } -// MergeSeriesSets returns a new series set that is the union of the input sets. +// MergeSeriesSets takes all series sets and returns as a union single series set. +// It assumes series are sorted by labels within single SeriesSet, similar to remote read guarantees. +// However, they can be partial: in such case, if the single SeriesSet returns the same series within many iterations, +// MergeSeriesSets will merge those into one. +// +// It also assumes in a "best effort" way that chunks are sorted by min time. It's done as an optimization only, so if input +// series' chunks are NOT sorted, the only consequence is that the duplicates might be not correctly removed. This is double checked +// which on just-before PromQL level as well, so the only consequence is increased network bandwidth. +// If all chunks were sorted, MergeSeriesSet ALSO returns sorted chunks by min time. +// +// Chunks within the same series can also overlap (within all SeriesSet +// as well as single SeriesSet alone). If the chunk ranges overlap, the *exact* chunk duplicates will be removed +// (except one), and any other overlaps will be appended into on chunks slice. func MergeSeriesSets(all ...SeriesSet) SeriesSet { switch len(all) { case 0: return emptySeriesSet{} case 1: - return all[0] + return newUniqueSeriesSet(all[0]) } h := len(all) / 2 @@ -106,11 +120,6 @@ type mergedSeriesSet struct { adone, bdone bool } -// newMergedSeriesSet takes two series sets as a single series set. -// Series that occur in both sets should have disjoint time ranges. -// If the ranges overlap b samples are appended to a samples. -// If the single SeriesSet returns same series within many iterations, -// merge series set will not try to merge those. func newMergedSeriesSet(a, b SeriesSet) *mergedSeriesSet { s := &mergedSeriesSet{a: a, b: b} // Initialize first elements of both sets as Next() needs @@ -150,33 +159,175 @@ func (s *mergedSeriesSet) Next() bool { } d := s.compare() - - // Both sets contain the current series. Chain them into a single one. if d > 0 { s.lset, s.chunks = s.b.At() s.bdone = !s.b.Next() - } else if d < 0 { + return true + } + if d < 0 { s.lset, s.chunks = s.a.At() s.adone = !s.a.Next() - } else { - // Concatenate chunks from both series sets. They may be expected of order - // w.r.t to their time range. This must be accounted for later. - lset, chksA := s.a.At() - _, chksB := s.b.At() - - s.lset = lset - // Slice reuse is not generally safe with nested merge iterators. - // We err on the safe side an create a new slice. - s.chunks = make([]AggrChunk, 0, len(chksA)+len(chksB)) - s.chunks = append(s.chunks, chksA...) - s.chunks = append(s.chunks, chksB...) + return true + } - s.adone = !s.a.Next() - s.bdone = !s.b.Next() + // Both a and b contains the same series. Go through all chunks, remove duplicates and concatenate chunks from both + // series sets. We best effortly assume chunks are sorted by min time. If not, we will not detect all deduplicate which will + // be account on select layer anyway. We do it still for early optimization. + lset, chksA := s.a.At() + _, chksB := s.b.At() + s.lset = lset + + // Slice reuse is not generally safe with nested merge iterators. + // We err on the safe side an create a new slice. + s.chunks = make([]AggrChunk, 0, len(chksA)+len(chksB)) + + b := 0 +Outer: + for a := range chksA { + for { + if b >= len(chksB) { + // No more b chunks. + s.chunks = append(s.chunks, chksA[a:]...) + break Outer + } + + cmp := chksA[a].Compare(chksB[b]) + if cmp > 0 { + s.chunks = append(s.chunks, chksA[a]) + break + } + if cmp < 0 { + s.chunks = append(s.chunks, chksB[b]) + b++ + continue + } + + // Exact duplicated chunks, discard one from b. + b++ + } } + + if b < len(chksB) { + s.chunks = append(s.chunks, chksB[b:]...) + } + + s.adone = !s.a.Next() + s.bdone = !s.b.Next() return true } +// uniqueSeriesSet takes one series set and ensures each iteration contains single, full series. +type uniqueSeriesSet struct { + SeriesSet + done bool + + peek *Series + + lset []Label + chunks []AggrChunk +} + +func newUniqueSeriesSet(wrapped SeriesSet) *uniqueSeriesSet { + return &uniqueSeriesSet{SeriesSet: wrapped} +} + +func (s *uniqueSeriesSet) At() ([]Label, []AggrChunk) { + return s.lset, s.chunks +} + +func (s *uniqueSeriesSet) Next() bool { + if s.Err() != nil { + return false + } + + for !s.done { + if s.done = !s.SeriesSet.Next(); s.done { + break + } + lset, chks := s.SeriesSet.At() + if s.peek == nil { + s.peek = &Series{Labels: lset, Chunks: chks} + continue + } + + if CompareLabels(lset, s.peek.Labels) != 0 { + s.lset, s.chunks = s.peek.Labels, s.peek.Chunks + s.peek = &Series{Labels: lset, Chunks: chks} + return true + } + + // We assume non-overlapping, sorted chunks. This is best effort only, if it's otherwise it + // will just be duplicated, but well handled by StoreAPI consumers. + s.peek.Chunks = append(s.peek.Chunks, chks...) + } + + if s.peek == nil { + return false + } + + s.lset, s.chunks = s.peek.Labels, s.peek.Chunks + s.peek = nil + return true +} + +// Compare returns positive 1 if chunk is smaller -1 if larger than b by min time, then max time. +// It returns 0 if chunks are exactly the same. +func (m AggrChunk) Compare(b AggrChunk) int { + if m.MinTime < b.MinTime { + return 1 + } + if m.MinTime > b.MinTime { + return -1 + } + + // Same min time. + if m.MaxTime < b.MaxTime { + return 1 + } + if m.MaxTime > b.MaxTime { + return -1 + } + + // We could use proto.Equal, but we need ordering as well. + for _, cmp := range []func() int{ + func() int { return m.Raw.Compare(b.Raw) }, + func() int { return m.Count.Compare(b.Count) }, + func() int { return m.Sum.Compare(b.Sum) }, + func() int { return m.Min.Compare(b.Min) }, + func() int { return m.Max.Compare(b.Max) }, + func() int { return m.Counter.Compare(b.Counter) }, + } { + if c := cmp(); c == 0 { + continue + } else { + return c + } + } + return 0 +} + +// Compare returns positive 1 if chunk is smaller -1 if larger. +// It returns 0 if chunks are exactly the same. +func (m *Chunk) Compare(b *Chunk) int { + if m == nil && b == nil { + return 0 + } + if b == nil { + return 1 + } + if m == nil { + return -1 + } + + if m.Type < b.Type { + return 1 + } + if m.Type > b.Type { + return -1 + } + return bytes.Compare(m.Data, b.Data) +} + // LabelsToPromLabels converts Thanos proto labels to Prometheus labels in type safe manner. func LabelsToPromLabels(lset []Label) labels.Labels { ret := make(labels.Labels, len(lset)) diff --git a/pkg/store/storepb/custom_test.go b/pkg/store/storepb/custom_test.go index cbeaed6950..832c6e0b9b 100644 --- a/pkg/store/storepb/custom_test.go +++ b/pkg/store/storepb/custom_test.go @@ -87,7 +87,7 @@ func (errSeriesSet) At() ([]Label, []AggrChunk) { return nil, nil } func (e errSeriesSet) Err() error { return e.err } -func TestMergeSeriesSet(t *testing.T) { +func TestMergeSeriesSets(t *testing.T) { for _, tcase := range []struct { desc string in [][]rawSeries @@ -139,7 +139,7 @@ func TestMergeSeriesSet(t *testing.T) { }, }, { - desc: "two seriesSets, {a=c} series to merge", + desc: "two seriesSets, {a=c} series to merge, sorted", in: [][]rawSeries{ { { @@ -165,14 +165,12 @@ func TestMergeSeriesSet(t *testing.T) { chunks: [][]sample{{{1, 1}, {2, 2}}, {{3, 3}, {4, 4}}}, }, { lset: labels.FromStrings("a", "c"), - chunks: [][]sample{{{11, 1}, {12, 2}}, {{13, 3}, {14, 4}}, {{7, 1}, {8, 2}}, {{9, 3}, {10, 4}, {11, 4444}}}, + chunks: [][]sample{{{7, 1}, {8, 2}}, {{9, 3}, {10, 4}, {11, 4444}}, {{11, 1}, {12, 2}}, {{13, 3}, {14, 4}}}, }, }, }, { - // SeriesSet can return same series within different iterations. MergeSeries should not try to merge those. - // We do it on last step possible: Querier promSet. - desc: "single seriesSets, {a=c} series to merge.", + desc: "single seriesSets, {a=c} series to merge, sorted", in: [][]rawSeries{ { { @@ -196,25 +194,146 @@ func TestMergeSeriesSet(t *testing.T) { chunks: [][]sample{{{1, 1}, {2, 2}}, {{3, 3}, {4, 4}}}, }, { lset: labels.FromStrings("a", "c"), - chunks: [][]sample{{{7, 1}, {8, 2}}, {{9, 3}, {10, 4}, {11, 4444}}}, + chunks: [][]sample{{{7, 1}, {8, 2}}, {{9, 3}, {10, 4}, {11, 4444}}, {{11, 1}, {12, 2}}, {{13, 3}, {14, 4}}}, + }, + }, + }, + { + desc: "four seriesSets, {a=c} series to merge AND deduplicate exactly the same chunks", + in: [][]rawSeries{ + { + { + lset: labels.FromStrings("a", "a"), + chunks: [][]sample{{{1, 1}, {2, 2}}, {{3, 3}, {4, 4}}}, + }, + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{11, 11}, {12, 12}, {13, 13}, {14, 14}}, + {{15, 15}, {16, 16}, {17, 17}, {18, 18}}, + }, + }, + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{20, 20}, {21, 21}, {22, 22}, {24, 24}}, + }, + }, + }, + { + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{1, 1}, {2, 2}, {3, 3}, {4, 4}}, + {{11, 11}, {12, 12}, {13, 13}, {14, 14}}, // Same chunk as in set 1. + }, + }, + { + lset: labels.FromStrings("a", "d"), + chunks: [][]sample{{{11, 1}, {12, 2}}, {{13, 3}, {14, 4}}}, + }, + }, + { + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{11, 11}, {12, 12}, {13, 13}, {14, 14}}, // Same chunk as in set 1. + {{20, 20}, {21, 21}, {22, 23}, {24, 24}}, // Almost same chunk as in set 1 (one value is different). + }, + }, + }, + { + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{11, 11}, {12, 12}, {14, 14}}, // Almost same chunk as in set 1 (one sample is missing). + {{20, 20}, {21, 21}, {22, 22}, {24, 24}}, // Same chunk as in set 1. + }, + }, + }, + }, + + expected: []rawSeries{ + { + lset: labels.Labels{labels.Label{Name: "a", Value: "a"}}, + chunks: [][]sample{{{t: 1, v: 1}, {t: 2, v: 2}}, {{t: 3, v: 3}, {t: 4, v: 4}}}, }, { - lset: labels.FromStrings("a", "c"), - chunks: [][]sample{{{11, 1}, {12, 2}}, {{13, 3}, {14, 4}}}, + lset: labels.Labels{labels.Label{Name: "a", Value: "c"}}, + chunks: [][]sample{ + {{t: 1, v: 1}, {t: 2, v: 2}, {t: 3, v: 3}, {t: 4, v: 4}}, + {{t: 11, v: 11}, {t: 12, v: 12}, {t: 13, v: 13}, {t: 14, v: 14}}, + {{t: 11, v: 11}, {t: 12, v: 12}, {t: 14, v: 14}}, + {{t: 15, v: 15}, {t: 16, v: 16}, {t: 17, v: 17}, {t: 18, v: 18}}, + {{t: 20, v: 20}, {t: 21, v: 21}, {t: 22, v: 22}, {t: 24, v: 24}}, + {{t: 20, v: 20}, {t: 21, v: 21}, {t: 22, v: 23}, {t: 24, v: 24}}, + }, + }, { + lset: labels.Labels{labels.Label{Name: "a", Value: "d"}}, + chunks: [][]sample{{{t: 11, v: 1}, {t: 12, v: 2}}, {{t: 13, v: 3}, {t: 14, v: 4}}}, + }, + }, + }, + { + desc: "four seriesSets, {a=c} series to merge, unsorted chunks, so dedup is expected to not be fully done", + in: [][]rawSeries{ + { + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{20, 20}, {21, 21}, {22, 22}, {24, 24}}, + }, + }, + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{11, 11}, {12, 12}, {13, 13}, {14, 14}}, + {{15, 15}, {16, 16}, {17, 17}, {18, 18}}, + }, + }, + }, + { + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{11, 11}, {12, 12}, {13, 13}, {14, 14}}, // Same chunk as in set 1. + {{1, 1}, {2, 2}, {3, 3}, {4, 4}}, + }, + }, + }, + { + { + lset: labels.FromStrings("a", "c"), + chunks: [][]sample{ + {{20, 20}, {21, 21}, {22, 23}, {24, 24}}, // Almost same chunk as in set 1 (one value is different). + {{11, 11}, {12, 12}, {13, 13}, {14, 14}}, // Same chunk as in set 1. + }, + }, + }, + }, + + expected: []rawSeries{ + { + lset: labels.Labels{labels.Label{Name: "a", Value: "c"}}, + chunks: [][]sample{ + {{t: 11, v: 11}, {t: 12, v: 12}, {t: 13, v: 13}, {t: 14, v: 14}}, + {{t: 1, v: 1}, {t: 2, v: 2}, {t: 3, v: 3}, {t: 4, v: 4}}, + {{t: 20, v: 20}, {t: 21, v: 21}, {t: 22, v: 22}, {t: 24, v: 24}}, + {{t: 11, v: 11}, {t: 12, v: 12}, {t: 13, v: 13}, {t: 14, v: 14}}, + {{t: 15, v: 15}, {t: 16, v: 16}, {t: 17, v: 17}, {t: 18, v: 18}}, + {{t: 20, v: 20}, {t: 21, v: 21}, {t: 22, v: 23}, {t: 24, v: 24}}, + {{t: 11, v: 11}, {t: 12, v: 12}, {t: 13, v: 13}, {t: 14, v: 14}}, + }, }, }, }, } { - if ok := t.Run(tcase.desc, func(t *testing.T) { + t.Run(tcase.desc, func(t *testing.T) { var input []SeriesSet for _, iss := range tcase.in { input = append(input, newListSeriesSet(t, iss)) } - ss := MergeSeriesSets(input...) - seriesEquals(t, tcase.expected, ss) - testutil.Ok(t, ss.Err()) - }); !ok { - return - } + testutil.Equals(t, tcase.expected, expandSeriesSet(t, MergeSeriesSets(input...))) + }) } } @@ -239,42 +358,48 @@ type rawSeries struct { chunks [][]sample } -func seriesEquals(t *testing.T, expected []rawSeries, gotSS SeriesSet) { - var got []Series +func expandSeriesSet(t *testing.T, gotSS SeriesSet) (ret []rawSeries) { for gotSS.Next() { lset, chks := gotSS.At() - got = append(got, Series{Labels: lset, Chunks: chks}) - } - - testutil.Equals(t, len(expected), len(got), "got: %v", got) - - for i, series := range got { - testutil.Equals(t, expected[i].lset, LabelsToPromLabels(series.Labels)) - testutil.Equals(t, len(expected[i].chunks), len(series.Chunks), "unexpected number of chunks") - for k, chk := range series.Chunks { + r := rawSeries{lset: LabelsToPromLabels(lset), chunks: make([][]sample, len(chks))} + for i, chk := range chks { c, err := chunkenc.FromData(chunkenc.EncXOR, chk.Raw.Data) testutil.Ok(t, err) - j := 0 iter := c.Iterator(nil) for iter.Next() { - testutil.Assert(t, j < len(expected[i].chunks[k]), "more samples than expected for %s chunk %d", series.Labels, k) - - tv, v := iter.At() - testutil.Equals(t, expected[i].chunks[k][j], sample{tv, v}) - j++ + t, v := iter.At() + r.chunks[i] = append(r.chunks[i], sample{t: t, v: v}) } testutil.Ok(t, iter.Err()) - testutil.Equals(t, len(expected[i].chunks[k]), j) } + ret = append(ret, r) } - + testutil.Ok(t, gotSS.Err()) + return ret } // Test the cost of merging series sets for different number of merged sets and their size. -// The subset are all equivalent so this does not capture merging of partial or non-overlapping sets well. func BenchmarkMergedSeriesSet(b *testing.B) { + b.Run("overlapping chunks", func(b *testing.B) { + benchmarkMergedSeriesSet(testutil.NewTB(b), true) + }) + b.Run("non-overlapping chunks", func(b *testing.B) { + benchmarkMergedSeriesSet(testutil.NewTB(b), false) + }) +} + +func TestMergedSeriesSet_Labels(t *testing.T) { + t.Run("overlapping chunks", func(t *testing.T) { + benchmarkMergedSeriesSet(testutil.NewTB(t), true) + }) + t.Run("non-overlapping chunks", func(t *testing.T) { + benchmarkMergedSeriesSet(testutil.NewTB(t), false) + }) +} + +func benchmarkMergedSeriesSet(b testutil.TB, overlappingChunks bool) { var sel func(sets []SeriesSet) SeriesSet sel = func(sets []SeriesSet) SeriesSet { if len(sets) == 0 { @@ -295,25 +420,34 @@ func BenchmarkMergedSeriesSet(b *testing.B) { 20000, } { for _, j := range []int{1, 2, 4, 8, 16, 32} { - b.Run(fmt.Sprintf("series=%d,blocks=%d", k, j), func(b *testing.B) { + b.Run(fmt.Sprintf("series=%d,blocks=%d", k, j), func(b testutil.TB) { lbls, err := labels.ReadLabels(filepath.Join("../../testutil/testdata", "20kseries.json"), k) testutil.Ok(b, err) sort.Sort(labels.Slice(lbls)) - in := make([][]rawSeries, j) - + blocks := make([][]rawSeries, j) for _, l := range lbls { - for j := range in { - in[j] = append(in[j], rawSeries{lset: l, chunks: chunks}) + for j := range blocks { + if overlappingChunks { + blocks[j] = append(blocks[j], rawSeries{lset: l, chunks: chunks}) + continue + } + blocks[j] = append(blocks[j], rawSeries{ + lset: l, + chunks: [][]sample{ + {{int64(4*j) + 1, 1}, {int64(4*j) + 2, 2}}, + {{int64(4*j) + 3, 3}, {int64(4*j) + 4, 4}}, + }, + }) } } b.ResetTimer() - for i := 0; i < b.N; i++ { + for i := 0; i < b.N(); i++ { var sets []SeriesSet - for _, s := range in { + for _, s := range blocks { sets = append(sets, newListSeriesSet(b, s)) } ms := sel(sets) diff --git a/pkg/store/storepb/rpc.pb.go b/pkg/store/storepb/rpc.pb.go index 7f1ee6a5eb..8ed700da80 100644 --- a/pkg/store/storepb/rpc.pb.go +++ b/pkg/store/storepb/rpc.pb.go @@ -738,6 +738,9 @@ type StoreClient interface { /// partition of the single series, but once a new series is started to be streamed it means that no more data will /// be sent for previous one. /// Series has to be sorted. + /// + /// There is no requirements on chunk sorting, however it is recommended to have chunk sorted by chunk min time. + /// This heavily optimizes the resource usage on Querier / Federated Queries. Series(ctx context.Context, in *SeriesRequest, opts ...grpc.CallOption) (Store_SeriesClient, error) /// LabelNames returns all label names that is available. /// Currently unimplemented in all Thanos implementations, because Query API does not implement this either. @@ -824,6 +827,9 @@ type StoreServer interface { /// partition of the single series, but once a new series is started to be streamed it means that no more data will /// be sent for previous one. /// Series has to be sorted. + /// + /// There is no requirements on chunk sorting, however it is recommended to have chunk sorted by chunk min time. + /// This heavily optimizes the resource usage on Querier / Federated Queries. Series(*SeriesRequest, Store_SeriesServer) error /// LabelNames returns all label names that is available. /// Currently unimplemented in all Thanos implementations, because Query API does not implement this either. diff --git a/pkg/store/storepb/rpc.proto b/pkg/store/storepb/rpc.proto index 9d1ca8623c..3f96f9c2f5 100644 --- a/pkg/store/storepb/rpc.proto +++ b/pkg/store/storepb/rpc.proto @@ -34,6 +34,9 @@ service Store { /// partition of the single series, but once a new series is started to be streamed it means that no more data will /// be sent for previous one. /// Series has to be sorted. + /// + /// There is no requirements on chunk sorting, however it is recommended to have chunk sorted by chunk min time. + /// This heavily optimizes the resource usage on Querier / Federated Queries. rpc Series(SeriesRequest) returns (stream SeriesResponse); /// LabelNames returns all label names that is available.