From 1f7b7f7f1cc039271c219aaaefca94ea3b04024c Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Wed, 4 Jan 2023 19:37:30 +0100 Subject: [PATCH 1/8] Added native histogram support for sidecar, query, receiver. Signed-off-by: Sebastian Rabenhorst Added comment Signed-off-by: Sebastian Rabenhorst Cleanup native histogram tests Signed-off-by: Sebastian Rabenhorst Made native hist test ha and replace usage of at with att Signed-off-by: Sebastian Rabenhorst Fix deduplication with native histograms Signed-off-by: Sebastian Rabenhorst Enabled native histogram writing Signed-off-by: Sebastian Rabenhorst Committed missing files for write Signed-off-by: Sebastian Rabenhorst Added benchmark for histogram writing Signed-off-by: Sebastian Rabenhorst Fixed chunkSeriesIterator Signed-off-by: Sebastian Rabenhorst Enabled replication for native histogram write test Signed-off-by: Sebastian Rabenhorst Lint native histogram test imports Signed-off-by: Sebastian Rabenhorst Removed ToDo comments Signed-off-by: Sebastian Rabenhorst Cleanup Signed-off-by: Sebastian Rabenhorst --- cmd/thanos/receive.go | 2 + internal/cortex/querier/series/series_set.go | 4 +- .../cortex/querier/series/series_set_test.go | 2 +- pkg/dedup/iter.go | 87 +- pkg/dedup/iter_test.go | 13 +- pkg/dedup/pushdown_iter.go | 5 +- pkg/promclient/promclient.go | 24 +- pkg/query/iter.go | 29 +- pkg/query/querier_test.go | 1 - pkg/receive/head_series_limiter.go | 2 +- pkg/receive/writer.go | 24 + pkg/receive/writer_test.go | 173 +- pkg/store/prometheus.go | 2 +- pkg/store/storepb/custom.go | 28 + pkg/store/storepb/prompb/types.pb.go | 1652 +++++++++++++++-- pkg/store/storepb/prompb/types.proto | 68 + pkg/store/storepb/types.pb.go | 78 +- pkg/store/storepb/types.proto | 1 + test/e2e/native_histograms_test.go | 159 +- test/e2e/query_test.go | 4 +- 20 files changed, 2056 insertions(+), 302 deletions(-) diff --git a/cmd/thanos/receive.go b/cmd/thanos/receive.go index b312ec4b5f..e5dcc611ad 100644 --- a/cmd/thanos/receive.go +++ b/cmd/thanos/receive.go @@ -88,6 +88,8 @@ func registerReceive(app *extkingpin.App) { EnableExemplarStorage: true, HeadChunksWriteQueueSize: int(conf.tsdbWriteQueueSize), EnableMemorySnapshotOnShutdown: conf.tsdbMemorySnapshotOnShutdown, + //TODO (rabenhorst): Do we want to make this configurable? + EnableNativeHistograms: true, } // Are we running in IngestorOnly, RouterOnly or RouterIngestor mode? diff --git a/internal/cortex/querier/series/series_set.go b/internal/cortex/querier/series/series_set.go index d86d3d1f1e..ff43361b96 100644 --- a/internal/cortex/querier/series/series_set.go +++ b/internal/cortex/querier/series/series_set.go @@ -239,7 +239,7 @@ func (d DeletedSeriesIterator) Seek(t int64) chunkenc.ValueType { return valueType } - seekedTs, _ := d.itr.At() + seekedTs := d.itr.AtT() if d.isDeleted(seekedTs) { // point we have seeked into is deleted, Next() should find a new non-deleted sample which is after t and seekedTs return d.Next() @@ -268,7 +268,7 @@ func (d DeletedSeriesIterator) AtT() int64 { func (d DeletedSeriesIterator) Next() chunkenc.ValueType { for valueType := d.itr.Next(); valueType != chunkenc.ValNone; valueType = d.itr.Next() { - ts, _ := d.itr.At() + ts := d.itr.AtT() if d.isDeleted(ts) { continue diff --git a/internal/cortex/querier/series/series_set_test.go b/internal/cortex/querier/series/series_set_test.go index 844d9799a8..4210ba832e 100644 --- a/internal/cortex/querier/series/series_set_test.go +++ b/internal/cortex/querier/series/series_set_test.go @@ -145,7 +145,7 @@ func TestDeletedIterator_WithSeek(t *testing.T) { require.Equal(t, c.valueType, it.Seek(c.seek)) if c.valueType != chunkenc.ValNone { - ts, _ := it.At() + ts := it.AtT() require.Equal(t, c.seekedTs, ts) } } diff --git a/pkg/dedup/iter.go b/pkg/dedup/iter.go index 0aa76651cc..2462b3c3b6 100644 --- a/pkg/dedup/iter.go +++ b/pkg/dedup/iter.go @@ -380,8 +380,8 @@ type dedupSeriesIterator struct { // TODO(bwplotka): Don't base on LastT, but on detected scrape interval. This will allow us to be more // responsive to gaps: https://github.com/thanos-io/thanos/issues/981, let's do it in next PR. - lastT int64 - lastV float64 + lastT int64 + lastIter chunkenc.Iterator penA, penB int64 useA bool @@ -389,23 +389,23 @@ type dedupSeriesIterator struct { func newDedupSeriesIterator(a, b adjustableSeriesIterator) *dedupSeriesIterator { return &dedupSeriesIterator{ - a: a, - b: b, - lastT: math.MinInt64, - lastV: float64(math.MinInt64), - aval: a.Next(), - bval: b.Next(), + a: a, + b: b, + lastT: math.MinInt64, + lastIter: a, + aval: a.Next(), + bval: b.Next(), } } func (it *dedupSeriesIterator) Next() chunkenc.ValueType { - lastValue := it.lastV + lastFloatVal, isFloatVal := it.lastFloatVal() lastUseA := it.useA defer func() { - if it.useA != lastUseA { + if it.useA != lastUseA && isFloatVal { // We switched replicas. // Ensure values are correct bases on value before At. - it.adjustAtValue(lastValue) + it.adjustAtValue(lastFloatVal) } }() @@ -421,14 +421,16 @@ func (it *dedupSeriesIterator) Next() chunkenc.ValueType { if it.aval == chunkenc.ValNone { it.useA = false if it.bval != chunkenc.ValNone { - it.lastT, it.lastV = it.b.At() + it.lastT = it.b.AtT() + it.lastIter = it.b it.penB = 0 } return it.bval } if it.bval == chunkenc.ValNone { it.useA = true - it.lastT, it.lastV = it.a.At() + it.lastT = it.a.AtT() + it.lastIter = it.a it.penA = 0 return it.aval } @@ -436,8 +438,8 @@ func (it *dedupSeriesIterator) Next() chunkenc.ValueType { // with the smaller timestamp. // The applied penalty potentially already skipped potential samples already // that would have resulted in exaggerated sampling frequency. - ta, va := it.a.At() - tb, vb := it.b.At() + ta := it.a.AtT() + tb := it.b.AtT() it.useA = ta <= tb @@ -458,7 +460,8 @@ func (it *dedupSeriesIterator) Next() chunkenc.ValueType { } it.penA = 0 it.lastT = ta - it.lastV = va + it.lastIter = it.a + return it.aval } if it.lastT != math.MinInt64 { @@ -468,26 +471,40 @@ func (it *dedupSeriesIterator) Next() chunkenc.ValueType { } it.penB = 0 it.lastT = tb - it.lastV = vb + it.lastIter = it.b return it.bval } +func (it *dedupSeriesIterator) lastFloatVal() (float64, bool) { + if it.useA && it.aval == chunkenc.ValFloat { + _, v := it.lastIter.At() + return v, true + } + if !it.useA && it.bval == chunkenc.ValFloat { + _, v := it.lastIter.At() + return v, true + } + return 0, false +} + func (it *dedupSeriesIterator) adjustAtValue(lastValue float64) { - if it.aval != chunkenc.ValNone { + if it.aval == chunkenc.ValFloat { it.a.adjustAtValue(lastValue) } - if it.bval != chunkenc.ValNone { + if it.bval == chunkenc.ValFloat { it.b.adjustAtValue(lastValue) } } -// TODO(rabenhorst): Native histogram support needs to be implemented, float type hardcoded. func (it *dedupSeriesIterator) Seek(t int64) chunkenc.ValueType { // Don't use underlying Seek, but iterate over next to not miss gaps. for { - ts, _ := it.At() + ts := it.AtT() if ts >= t { - return chunkenc.ValFloat + if it.useA { + return it.a.Seek(ts) + } + return it.b.Seek(ts) } if it.Next() == chunkenc.ValNone { return chunkenc.ValNone @@ -496,27 +513,23 @@ func (it *dedupSeriesIterator) Seek(t int64) chunkenc.ValueType { } func (it *dedupSeriesIterator) At() (int64, float64) { - if it.useA { - return it.a.At() - } - return it.b.At() + return it.lastIter.At() } -// TODO(rabenhorst): Needs to be implemented for native histogram support. func (it *dedupSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { - panic("not implemented") + return it.lastIter.AtHistogram() } func (it *dedupSeriesIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) { - panic("not implemented") + return it.lastIter.AtFloatHistogram() } func (it *dedupSeriesIterator) AtT() int64 { var t int64 if it.useA { - t, _ = it.a.At() + t = it.a.AtT() } else { - t, _ = it.b.At() + t = it.b.AtT() } return t } @@ -553,18 +566,16 @@ func (it *boundedSeriesIterator) At() (t int64, v float64) { return it.it.At() } -// TODO(rabenhorst): Needs to be implemented for native histogram support. func (it *boundedSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { - panic("not implemented") + return it.it.AtHistogram() } func (it *boundedSeriesIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) { - panic("not implemented") + return it.it.AtFloatHistogram() } func (it *boundedSeriesIterator) AtT() int64 { - t, _ := it.it.At() - return t + return it.it.AtT() } func (it *boundedSeriesIterator) Next() chunkenc.ValueType { @@ -572,14 +583,14 @@ func (it *boundedSeriesIterator) Next() chunkenc.ValueType { if valueType == chunkenc.ValNone { return chunkenc.ValNone } - t, _ := it.it.At() + t := it.it.AtT() // Advance the iterator if we are before the valid interval. if t < it.mint { if it.Seek(it.mint) == chunkenc.ValNone { return chunkenc.ValNone } - t, _ = it.it.At() + t = it.it.AtT() } // Once we passed the valid interval, there is no going back. if t <= it.maxt { diff --git a/pkg/dedup/iter_test.go b/pkg/dedup/iter_test.go index cdf16bb96e..2ecd74473f 100644 --- a/pkg/dedup/iter_test.go +++ b/pkg/dedup/iter_test.go @@ -147,13 +147,14 @@ var expectedRealSeriesWithStaleMarkerDeduplicatedForRate = []sample{ func TestDedupSeriesSet(t *testing.T) { tests := []struct { + name string input []series exp []series dedupLabels map[string]struct{} isCounter bool }{ { - // Single dedup label. + name: "Single dedup label", input: []series{ { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}}, @@ -210,6 +211,7 @@ func TestDedupSeriesSet(t *testing.T) { { // 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"}}, @@ -238,7 +240,7 @@ func TestDedupSeriesSet(t *testing.T) { dedupLabels: map[string]struct{}{"replica": {}, "replica2": {}, "replica3": {}, "replica4": {}, "replica5": {}, "replica6": {}, "replica7": {}}, }, { - // Multi dedup label. + 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"}}, @@ -294,7 +296,7 @@ func TestDedupSeriesSet(t *testing.T) { }, }, { - // Multi dedup label - some series don't have all dedup labels. + 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"}}, @@ -322,6 +324,7 @@ func TestDedupSeriesSet(t *testing.T) { // Now, depending on what replica we look, we can see totally different counter value in total where total means // after accounting for counter resets. We account for that in downsample.CounterSeriesIterator, mainly because // we handle downsample Counter Aggregations specially (for detecting resets between chunks). + name: "Regression test against 2401", isCounter: true, input: []series{ { @@ -362,6 +365,7 @@ func TestDedupSeriesSet(t *testing.T) { }, { // Same thing but not for counter should not adjust anything. + name: "Regression test with no counter adjustment", isCounter: false, input: []series{ { @@ -387,6 +391,7 @@ func TestDedupSeriesSet(t *testing.T) { { // Regression test on real data against https://github.com/thanos-io/thanos/issues/2401. // Real data with stale marker after downsample.CounterSeriesIterator (required for downsampling + rate). + name: "Regression test on real data against 2401", isCounter: true, input: []series{ { @@ -456,7 +461,7 @@ func TestDedupSeriesSet(t *testing.T) { } for _, tcase := range tests { - t.Run("", func(t *testing.T) { + 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 { diff --git a/pkg/dedup/pushdown_iter.go b/pkg/dedup/pushdown_iter.go index 00c970b929..76f8958e79 100644 --- a/pkg/dedup/pushdown_iter.go +++ b/pkg/dedup/pushdown_iter.go @@ -122,14 +122,13 @@ func (it *pushdownSeriesIterator) AtFloatHistogram() (int64, *histogram.FloatHis } func (it *pushdownSeriesIterator) AtT() int64 { - t, _ := it.a.At() + t := it.a.AtT() return t } -// TODO(rabenhorst): Native histogram support needs to be implemented, currently float type is hardcoded. func (it *pushdownSeriesIterator) Seek(t int64) chunkenc.ValueType { for { - ts, _ := it.At() + ts := it.AtT() if ts >= t { return chunkenc.ValFloat } diff --git a/pkg/promclient/promclient.go b/pkg/promclient/promclient.go index 748ebf189b..13e9f71eb7 100644 --- a/pkg/promclient/promclient.go +++ b/pkg/promclient/promclient.go @@ -385,15 +385,15 @@ func (p *QueryOptions) AddTo(values url.Values) error { } // QueryInstant performs an instant query using a default HTTP client and returns results in model.Vector type. -func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, t time.Time, opts QueryOptions) (model.Vector, []string, error) { +func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, t time.Time, opts QueryOptions) (model.Vector, json.RawMessage, []string, error) { params, err := url.ParseQuery(base.RawQuery) if err != nil { - return nil, nil, errors.Wrapf(err, "parse raw query %s", base.RawQuery) + return nil, nil, nil, errors.Wrapf(err, "parse raw query %s", base.RawQuery) } params.Add("query", query) params.Add("time", t.Format(time.RFC3339Nano)) if err := opts.AddTo(params); err != nil { - return nil, nil, errors.Wrap(err, "add thanos opts query params") + return nil, nil, nil, errors.Wrap(err, "add thanos opts query params") } u := *base @@ -412,7 +412,7 @@ func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, body, _, err := c.req2xx(ctx, &u, method) if err != nil { - return nil, nil, errors.Wrap(err, "read query instant response") + return nil, nil, nil, errors.Wrap(err, "read query instant response") } // Decode only ResultType and load Result only as RawJson since we don't know @@ -430,7 +430,7 @@ func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, } if err = json.Unmarshal(body, &m); err != nil { - return nil, nil, errors.Wrap(err, "unmarshal query instant response") + return nil, nil, nil, errors.Wrap(err, "unmarshal query instant response") } var vectorResult model.Vector @@ -440,29 +440,29 @@ func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, switch m.Data.ResultType { case string(parser.ValueTypeVector): if err = json.Unmarshal(m.Data.Result, &vectorResult); err != nil { - return nil, nil, errors.Wrap(err, "decode result into ValueTypeVector") + return nil, nil, nil, errors.Wrap(err, "decode result into ValueTypeVector") } case string(parser.ValueTypeScalar): vectorResult, err = convertScalarJSONToVector(m.Data.Result) if err != nil { - return nil, nil, errors.Wrap(err, "decode result into ValueTypeScalar") + return nil, nil, nil, errors.Wrap(err, "decode result into ValueTypeScalar") } default: if m.Warnings != nil { - return nil, nil, errors.Errorf("error: %s, type: %s, warning: %s", m.Error, m.ErrorType, strings.Join(m.Warnings, ", ")) + return nil, nil, nil, errors.Errorf("error: %s, type: %s, warning: %s", m.Error, m.ErrorType, strings.Join(m.Warnings, ", ")) } if m.Error != "" { - return nil, nil, errors.Errorf("error: %s, type: %s", m.Error, m.ErrorType) + return nil, nil, nil, errors.Errorf("error: %s, type: %s", m.Error, m.ErrorType) } - return nil, nil, errors.Errorf("received status code: 200, unknown response type: '%q'", m.Data.ResultType) + return nil, nil, nil, errors.Errorf("received status code: 200, unknown response type: '%q'", m.Data.ResultType) } - return vectorResult, m.Warnings, nil + return vectorResult, m.Data.Result, m.Warnings, nil } // PromqlQueryInstant performs instant query and returns results in promql.Vector type that is compatible with promql package. func (c *Client) PromqlQueryInstant(ctx context.Context, base *url.URL, query string, t time.Time, opts QueryOptions) (promql.Vector, []string, error) { - vectorResult, warnings, err := c.QueryInstant(ctx, base, query, t, opts) + vectorResult, _, warnings, err := c.QueryInstant(ctx, base, query, t, opts) if err != nil { return nil, nil, err } diff --git a/pkg/query/iter.go b/pkg/query/iter.go index e0fb8e2c12..e01aeb49c2 100644 --- a/pkg/query/iter.go +++ b/pkg/query/iter.go @@ -238,6 +238,8 @@ func chunkEncoding(e storepb.Chunk_Encoding) chunkenc.Encoding { switch e { case storepb.Chunk_XOR: return chunkenc.EncXOR + case storepb.Chunk_HISTOGRAM: + return chunkenc.EncHistogram } return 255 // Invalid. } @@ -257,8 +259,9 @@ func (it errSeriesIterator) Err() error { // chunkSeriesIterator implements a series iterator on top // of a list of time-sorted, non-overlapping chunks. type chunkSeriesIterator struct { - chunks []chunkenc.Iterator - i int + chunks []chunkenc.Iterator + i int + lastVal chunkenc.ValueType } func newChunkSeriesIterator(cs []chunkenc.Iterator) chunkenc.Iterator { @@ -269,17 +272,17 @@ func newChunkSeriesIterator(cs []chunkenc.Iterator) chunkenc.Iterator { return &chunkSeriesIterator{chunks: cs} } -// TODO(rabenhorst: Native histogram support needs to be implement, currently float type is hardcoded. func (it *chunkSeriesIterator) Seek(t int64) chunkenc.ValueType { // We generally expect the chunks already to be cut down // to the range we are interested in. There's not much to be gained from // hopping across chunks so we just call next until we reach t. for { - ct, _ := it.At() + ct := it.AtT() if ct >= t { - return chunkenc.ValFloat + return it.lastVal } - if it.Next() == chunkenc.ValNone { + it.lastVal = it.Next() + if it.lastVal == chunkenc.ValNone { return chunkenc.ValNone } } @@ -289,24 +292,24 @@ func (it *chunkSeriesIterator) At() (t int64, v float64) { return it.chunks[it.i].At() } -// TODO(rabenhorst): Needs to be implemented for native histogram support. func (it *chunkSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { - panic("not implemented") + return it.chunks[it.i].AtHistogram() } func (it *chunkSeriesIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) { - panic("not implemented") + return it.chunks[it.i].AtFloatHistogram() } func (it *chunkSeriesIterator) AtT() int64 { - t, _ := it.chunks[it.i].At() - return t + return it.chunks[it.i].AtT() } func (it *chunkSeriesIterator) Next() chunkenc.ValueType { - lastT, _ := it.At() + lastT := it.AtT() - if valueType := it.chunks[it.i].Next(); valueType != chunkenc.ValNone { + var valueType chunkenc.ValueType + if valueType = it.chunks[it.i].Next(); valueType != chunkenc.ValNone { + it.lastVal = valueType return valueType } if it.Err() != nil { diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 2789d72cce..3bb8cdba8b 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -843,7 +843,6 @@ func (s *mockedSeriesIterator) At() (t int64, v float64) { return sample.t, sample.v } -// TODO(rabenhorst): Needs to be implemented for native histogram support. func (s *mockedSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { panic("not implemented") } diff --git a/pkg/receive/head_series_limiter.go b/pkg/receive/head_series_limiter.go index 746b92d990..7536213ef9 100644 --- a/pkg/receive/head_series_limiter.go +++ b/pkg/receive/head_series_limiter.go @@ -106,7 +106,7 @@ func NewHeadSeriesLimit(w WriteLimitsConfig, registerer prometheus.Registerer, l func (h *headSeriesLimit) QueryMetaMonitoring(ctx context.Context) error { c := promclient.NewWithTracingClient(h.logger, h.metaMonitoringClient, httpconfig.ThanosUserAgent) - vectorRes, _, err := c.QueryInstant(ctx, h.metaMonitoringURL, h.metaMonitoringQuery, time.Now(), promclient.QueryOptions{}) + vectorRes, _, _, err := c.QueryInstant(ctx, h.metaMonitoringURL, h.metaMonitoringQuery, time.Now(), promclient.QueryOptions{}) if err != nil { h.metaMonitoringErr.Inc() return err diff --git a/pkg/receive/writer.go b/pkg/receive/writer.go index f59222f7ab..2aa2b757e5 100644 --- a/pkg/receive/writer.go +++ b/pkg/receive/writer.go @@ -14,6 +14,7 @@ import ( "github.com/prometheus/prometheus/tsdb" "github.com/thanos-io/thanos/pkg/store/labelpb" + "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/store/storepb/prompb" ) @@ -130,6 +131,29 @@ func (r *Writer) Write(ctx context.Context, tenantID string, wreq *prompb.WriteR } } + for _, hp := range t.Histograms { + h := storepb.HistogramProtoToHistogram(hp) + ref, err = app.AppendHistogram(ref, lset, hp.Timestamp, h) + switch err { + case storage.ErrOutOfOrderSample: + numSamplesOutOfOrder++ + level.Debug(tLogger).Log("msg", "Out of order histogram", "lset", lset, "timestamp", hp.Timestamp) + case storage.ErrDuplicateSampleForTimestamp: + numSamplesDuplicates++ + level.Debug(tLogger).Log("msg", "Duplicate histogram for timestamp", "lset", lset, "timestamp", hp.Timestamp) + case storage.ErrOutOfBounds: + numSamplesOutOfBounds++ + level.Debug(tLogger).Log("msg", "Out of bounds metric", "lset", lset, "timestamp", hp.Timestamp) + case storage.ErrTooOldSample: + numSamplesTooOld++ + level.Debug(tLogger).Log("msg", "Histogram is too old", "lset", lset, "timestamp", hp.Timestamp) + default: + if err != nil { + level.Debug(tLogger).Log("msg", "Error ingesting histogram", "err", err) + } + } + } + // Current implemetation of app.AppendExemplar doesn't create a new series, so it must be already present. // We drop the exemplars in case the series doesn't exist. if ref != 0 && len(t.Exemplars) > 0 { diff --git a/pkg/receive/writer_test.go b/pkg/receive/writer_test.go index 5760038028..b8aa4d7b98 100644 --- a/pkg/receive/writer_test.go +++ b/pkg/receive/writer_test.go @@ -14,6 +14,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" @@ -204,6 +205,62 @@ func TestWriter(t *testing.T) { expectedErr: errors.Wrapf(storage.ErrExemplarLabelLength, "add 1 exemplars"), maxExemplars: 2, }, + "should succeed on histogram with valid labels": { + reqs: []*prompb.WriteRequest{ + { + Timeseries: []prompb.TimeSeries{ + { + Labels: append(lbls, labelpb.ZLabel{Name: "a", Value: "1"}, labelpb.ZLabel{Name: "b", Value: "2"}), + Histograms: []prompb.Histogram{ + histogramToHistogramProto(9, testHistogram()), + }, + }, + }, + }, + }, + expectedErr: nil, + expectedIngested: []prompb.TimeSeries{ + { + Labels: append(lbls, labelpb.ZLabel{Name: "a", Value: "1"}, labelpb.ZLabel{Name: "b", Value: "2"}), + Histograms: []prompb.Histogram{ + histogramToHistogramProto(10, testHistogram()), + }, + }, + }, + }, + "should error out on valid histograms with out of order histogram": { + reqs: []*prompb.WriteRequest{ + { + Timeseries: []prompb.TimeSeries{ + { + Labels: append(lbls, labelpb.ZLabel{Name: "a", Value: "1"}, labelpb.ZLabel{Name: "b", Value: "2"}), + Histograms: []prompb.Histogram{ + histogramToHistogramProto(10, testHistogram()), + }, + }, + }, + }, + { + Timeseries: []prompb.TimeSeries{ + { + Labels: append(lbls, labelpb.ZLabel{Name: "a", Value: "1"}, labelpb.ZLabel{Name: "b", Value: "2"}), + Histograms: []prompb.Histogram{ + histogramToHistogramProto(9, testHistogram()), + }, + }, + }, + }, + }, + expectedErr: errors.Wrapf(storage.ErrOutOfOrderSample, "add 1 samples"), + expectedIngested: []prompb.TimeSeries{ + { + Labels: append(lbls, labelpb.ZLabel{Name: "a", Value: "1"}, labelpb.ZLabel{Name: "b", Value: "2"}), + Histograms: []prompb.Histogram{ + histogramToHistogramProto(10, testHistogram()), + }, + }, + }, + }, } for testName, testData := range tests { @@ -212,12 +269,13 @@ func TestWriter(t *testing.T) { logger := log.NewNopLogger() m := NewMultiTSDB(dir, logger, prometheus.NewRegistry(), &tsdb.Options{ - MinBlockDuration: (2 * time.Hour).Milliseconds(), - MaxBlockDuration: (2 * time.Hour).Milliseconds(), - RetentionDuration: (6 * time.Hour).Milliseconds(), - NoLockfile: true, - MaxExemplars: testData.maxExemplars, - EnableExemplarStorage: true, + MinBlockDuration: (2 * time.Hour).Milliseconds(), + MaxBlockDuration: (2 * time.Hour).Milliseconds(), + RetentionDuration: (6 * time.Hour).Milliseconds(), + NoLockfile: true, + MaxExemplars: testData.maxExemplars, + EnableExemplarStorage: true, + EnableNativeHistograms: true, }, labels.FromStrings("replica", "01"), "tenant_id", @@ -270,25 +328,46 @@ func TestWriter(t *testing.T) { } } -func BenchmarkWriterTimeSeriesWithSingleLabel_10(b *testing.B) { benchmarkWriter(b, 1, 10) } -func BenchmarkWriterTimeSeriesWithSingleLabel_100(b *testing.B) { benchmarkWriter(b, 1, 100) } -func BenchmarkWriterTimeSeriesWithSingleLabel_1000(b *testing.B) { benchmarkWriter(b, 1, 1000) } +func BenchmarkWriterTimeSeriesWithSingleLabel_10(b *testing.B) { benchmarkWriter(b, 1, 10, false) } +func BenchmarkWriterTimeSeriesWithSingleLabel_100(b *testing.B) { benchmarkWriter(b, 1, 100, false) } +func BenchmarkWriterTimeSeriesWithSingleLabel_1000(b *testing.B) { benchmarkWriter(b, 1, 1000, false) } -func BenchmarkWriterTimeSeriesWith10Labels_10(b *testing.B) { benchmarkWriter(b, 10, 10) } -func BenchmarkWriterTimeSeriesWith10Labels_100(b *testing.B) { benchmarkWriter(b, 10, 100) } -func BenchmarkWriterTimeSeriesWith10Labels_1000(b *testing.B) { benchmarkWriter(b, 10, 1000) } +func BenchmarkWriterTimeSeriesWith10Labels_10(b *testing.B) { benchmarkWriter(b, 10, 10, false) } +func BenchmarkWriterTimeSeriesWith10Labels_100(b *testing.B) { benchmarkWriter(b, 10, 100, false) } +func BenchmarkWriterTimeSeriesWith10Labels_1000(b *testing.B) { benchmarkWriter(b, 10, 1000, false) } -func benchmarkWriter(b *testing.B, labelsNum int, seriesNum int) { +func BenchmarkWriterTimeSeriesWithHistogramsWithSingleLabel_10(b *testing.B) { + benchmarkWriter(b, 1, 10, true) +} +func BenchmarkWriterTimeSeriesWithHistogramsWithSingleLabel_100(b *testing.B) { + benchmarkWriter(b, 1, 100, true) +} +func BenchmarkWriterTimeSeriesWithHistogramsWithSingleLabel_1000(b *testing.B) { + benchmarkWriter(b, 1, 1000, true) +} + +func BenchmarkWriterTimeSeriesWithHistogramsWith10Labels_10(b *testing.B) { + benchmarkWriter(b, 10, 10, true) +} +func BenchmarkWriterTimeSeriesWithHistogramsWith10Labels_100(b *testing.B) { + benchmarkWriter(b, 10, 100, true) +} +func BenchmarkWriterTimeSeriesWithHistogramsWith10Labels_1000(b *testing.B) { + benchmarkWriter(b, 10, 1000, true) +} + +func benchmarkWriter(b *testing.B, labelsNum int, seriesNum int, generateHistograms bool) { dir := b.TempDir() logger := log.NewNopLogger() m := NewMultiTSDB(dir, logger, prometheus.NewRegistry(), &tsdb.Options{ - MinBlockDuration: (2 * time.Hour).Milliseconds(), - MaxBlockDuration: (2 * time.Hour).Milliseconds(), - RetentionDuration: (6 * time.Hour).Milliseconds(), - NoLockfile: true, - MaxExemplars: 0, - EnableExemplarStorage: true, + MinBlockDuration: (2 * time.Hour).Milliseconds(), + MaxBlockDuration: (2 * time.Hour).Milliseconds(), + RetentionDuration: (6 * time.Hour).Milliseconds(), + NoLockfile: true, + MaxExemplars: 0, + EnableExemplarStorage: true, + EnableNativeHistograms: generateHistograms, }, labels.FromStrings("replica", "01"), "tenant_id", @@ -314,7 +393,7 @@ func benchmarkWriter(b *testing.B, labelsNum int, seriesNum int) { return err })) - timeSeries := generateLabelsAndSeries(labelsNum, seriesNum) + timeSeries := generateLabelsAndSeries(labelsNum, seriesNum, generateHistograms) wreq := &prompb.WriteRequest{ Timeseries: timeSeries, @@ -333,7 +412,7 @@ func benchmarkWriter(b *testing.B, labelsNum int, seriesNum int) { // duplicates without error (see comment https://github.com/prometheus/prometheus/blob/release-2.37/tsdb/head_append.go#L316). // This also means the sample won't be appended, which means the overhead of appending additional samples to head is not // reflected in the benchmark, but should still capture the performance of receive writer. -func generateLabelsAndSeries(numLabels int, numSeries int) []prompb.TimeSeries { +func generateLabelsAndSeries(numLabels int, numSeries int, generateHistograms bool) []prompb.TimeSeries { // Generate some labels first. l := make([]labelpb.ZLabel, 0, numLabels) l = append(l, labelpb.ZLabel{Name: "__name__", Value: "test"}) @@ -341,10 +420,58 @@ func generateLabelsAndSeries(numLabels int, numSeries int) []prompb.TimeSeries { l = append(l, labelpb.ZLabel{Name: fmt.Sprintf("label_%s", string(rune('a'+i))), Value: fmt.Sprintf("%d", i)}) } - ts := make([]prompb.TimeSeries, 0, numSeries) + ts := make([]prompb.TimeSeries, numSeries) for j := 0; j < numSeries; j++ { - ts = append(ts, prompb.TimeSeries{Labels: l, Samples: []prompb.Sample{{Value: 1, Timestamp: 10}}}) + ts[j] = prompb.TimeSeries{ + Labels: l, + } + + if generateHistograms { + ts[j].Histograms = []prompb.Histogram{histogramToHistogramProto(10, testHistogram())} + continue + } + + ts[j].Samples = []prompb.Sample{{Value: 1, Timestamp: 10}} } return ts } + +func testHistogram() *histogram.Histogram { + return &histogram.Histogram{ + Count: 5, + ZeroCount: 2, + Sum: 18.4, + ZeroThreshold: 0.1, + Schema: 1, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + PositiveBuckets: []int64{1, 1, -1, 0}, // counts: 1, 2, 1, 1 (total 5) + } +} + +func histogramToHistogramProto(timestamp int64, h *histogram.Histogram) prompb.Histogram { + return prompb.Histogram{ + Count: &prompb.Histogram_CountInt{CountInt: h.Count}, + Sum: h.Sum, + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: h.ZeroCount}, + NegativeSpans: spansToSpansProto(h.NegativeSpans), + NegativeDeltas: h.NegativeBuckets, + PositiveSpans: spansToSpansProto(h.PositiveSpans), + PositiveDeltas: h.PositiveBuckets, + Timestamp: timestamp, + } +} + +func spansToSpansProto(s []histogram.Span) []*prompb.BucketSpan { + spans := make([]*prompb.BucketSpan, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = &prompb.BucketSpan{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} diff --git a/pkg/store/prometheus.go b/pkg/store/prometheus.go index b4546c7d05..7dcb3baa14 100644 --- a/pkg/store/prometheus.go +++ b/pkg/store/prometheus.go @@ -255,7 +255,7 @@ func (p *PrometheusStore) queryPrometheus(s storepb.Store_SeriesServer, r *store } matrix = result } else { - vector, _, err := p.client.QueryInstant(s.Context(), p.base, r.ToPromQL(), timestamp.Time(r.MaxTime), opts) + vector, _, _, err := p.client.QueryInstant(s.Context(), p.base, r.ToPromQL(), timestamp.Time(r.MaxTime), opts) if err != nil { return err } diff --git a/pkg/store/storepb/custom.go b/pkg/store/storepb/custom.go index c1f4b9b8bf..85f858562b 100644 --- a/pkg/store/storepb/custom.go +++ b/pkg/store/storepb/custom.go @@ -13,10 +13,12 @@ import ( "github.com/gogo/protobuf/types" "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "google.golang.org/grpc/codes" "github.com/thanos-io/thanos/pkg/store/labelpb" + prompb "github.com/thanos-io/thanos/pkg/store/storepb/prompb" ) var PartialResponseStrategyValues = func() []string { @@ -552,3 +554,29 @@ func (m *QueryHints) IsSafeToExecute() bool { return false } + +// HistogramProtoToHistogram extracts a (normal integer) Histogram from the +// provided proto message. The caller has to make sure that the proto message +// represents an interger histogram and not a float histogram. +func HistogramProtoToHistogram(hp prompb.Histogram) *histogram.Histogram { + return &histogram.Histogram{ + Schema: hp.Schema, + ZeroThreshold: hp.ZeroThreshold, + ZeroCount: hp.GetZeroCountInt(), + Count: hp.GetCountInt(), + Sum: hp.Sum, + PositiveSpans: spansProtoToSpans(hp.GetPositiveSpans()), + PositiveBuckets: hp.GetPositiveDeltas(), + NegativeSpans: spansProtoToSpans(hp.GetNegativeSpans()), + NegativeBuckets: hp.GetNegativeDeltas(), + } +} + +func spansProtoToSpans(s []*prompb.BucketSpan) []histogram.Span { + spans := make([]histogram.Span, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = histogram.Span{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} diff --git a/pkg/store/storepb/prompb/types.pb.go b/pkg/store/storepb/prompb/types.pb.go index d8ef292dcf..f5332f345b 100644 --- a/pkg/store/storepb/prompb/types.pb.go +++ b/pkg/store/storepb/prompb/types.pb.go @@ -70,6 +70,37 @@ func (MetricMetadata_MetricType) EnumDescriptor() ([]byte, []int) { return fileDescriptor_166e07899dab7c14, []int{0, 0} } +type Histogram_ResetHint int32 + +const ( + Histogram_UNKNOWN Histogram_ResetHint = 0 + Histogram_YES Histogram_ResetHint = 1 + Histogram_NO Histogram_ResetHint = 2 + Histogram_GAUGE Histogram_ResetHint = 3 +) + +var Histogram_ResetHint_name = map[int32]string{ + 0: "UNKNOWN", + 1: "YES", + 2: "NO", + 3: "GAUGE", +} + +var Histogram_ResetHint_value = map[string]int32{ + "UNKNOWN": 0, + "YES": 1, + "NO": 2, + "GAUGE": 3, +} + +func (x Histogram_ResetHint) String() string { + return proto.EnumName(Histogram_ResetHint_name, int32(x)) +} + +func (Histogram_ResetHint) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_166e07899dab7c14, []int{3, 0} +} + type LabelMatcher_Type int32 const ( @@ -98,25 +129,28 @@ func (x LabelMatcher_Type) String() string { } func (LabelMatcher_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_166e07899dab7c14, []int{4, 0} + return fileDescriptor_166e07899dab7c14, []int{6, 0} } // We require this to match chunkenc.Encoding. type Chunk_Encoding int32 const ( - Chunk_UNKNOWN Chunk_Encoding = 0 - Chunk_XOR Chunk_Encoding = 1 + Chunk_UNKNOWN Chunk_Encoding = 0 + Chunk_XOR Chunk_Encoding = 1 + Chunk_HISTOGRAM Chunk_Encoding = 2 ) var Chunk_Encoding_name = map[int32]string{ 0: "UNKNOWN", 1: "XOR", + 2: "HISTOGRAM", } var Chunk_Encoding_value = map[string]int32{ - "UNKNOWN": 0, - "XOR": 1, + "UNKNOWN": 0, + "XOR": 1, + "HISTOGRAM": 2, } func (x Chunk_Encoding) String() string { @@ -124,7 +158,7 @@ func (x Chunk_Encoding) String() string { } func (Chunk_Encoding) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_166e07899dab7c14, []int{6, 0} + return fileDescriptor_166e07899dab7c14, []int{8, 0} } type MetricMetadata struct { @@ -305,20 +339,315 @@ func (m *Exemplar) GetTimestamp() int64 { return 0 } +// A native histogram, also known as a sparse histogram. +// Original design doc: +// https://docs.google.com/document/d/1cLNv3aufPZb3fNfaJgdaRBZsInZKKIHo9E6HinJVbpM/edit +// The appendix of this design doc also explains the concept of float +// histograms. This Histogram message can represent both, the usual +// integer histogram as well as a float histogram. +type Histogram struct { + // Types that are valid to be assigned to Count: + // + // *Histogram_CountInt + // *Histogram_CountFloat + Count isHistogram_Count `protobuf_oneof:"count"` + Sum float64 `protobuf:"fixed64,3,opt,name=sum,proto3" json:"sum,omitempty"` + // The schema defines the bucket schema. Currently, valid numbers + // are -4 <= n <= 8. They are all for base-2 bucket schemas, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n logarithmic buckets. Or in other words, each + // bucket boundary is the previous boundary times 2^(2^-n). In the + // future, more bucket schemas may be added using numbers < -4 or > + // 8. + Schema int32 `protobuf:"zigzag32,4,opt,name=schema,proto3" json:"schema,omitempty"` + ZeroThreshold float64 `protobuf:"fixed64,5,opt,name=zero_threshold,json=zeroThreshold,proto3" json:"zero_threshold,omitempty"` + // Types that are valid to be assigned to ZeroCount: + // + // *Histogram_ZeroCountInt + // *Histogram_ZeroCountFloat + ZeroCount isHistogram_ZeroCount `protobuf_oneof:"zero_count"` + // Negative Buckets. + NegativeSpans []*BucketSpan `protobuf:"bytes,8,rep,name=negative_spans,json=negativeSpans,proto3" json:"negative_spans,omitempty"` + // Use either "negative_deltas" or "negative_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + NegativeDeltas []int64 `protobuf:"zigzag64,9,rep,packed,name=negative_deltas,json=negativeDeltas,proto3" json:"negative_deltas,omitempty"` + NegativeCounts []float64 `protobuf:"fixed64,10,rep,packed,name=negative_counts,json=negativeCounts,proto3" json:"negative_counts,omitempty"` + // Positive Buckets. + PositiveSpans []*BucketSpan `protobuf:"bytes,11,rep,name=positive_spans,json=positiveSpans,proto3" json:"positive_spans,omitempty"` + // Use either "positive_deltas" or "positive_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + PositiveDeltas []int64 `protobuf:"zigzag64,12,rep,packed,name=positive_deltas,json=positiveDeltas,proto3" json:"positive_deltas,omitempty"` + PositiveCounts []float64 `protobuf:"fixed64,13,rep,packed,name=positive_counts,json=positiveCounts,proto3" json:"positive_counts,omitempty"` + ResetHint Histogram_ResetHint `protobuf:"varint,14,opt,name=reset_hint,json=resetHint,proto3,enum=prometheus_copy.Histogram_ResetHint" json:"reset_hint,omitempty"` + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + Timestamp int64 `protobuf:"varint,15,opt,name=timestamp,proto3" json:"timestamp,omitempty"` +} + +func (m *Histogram) Reset() { *m = Histogram{} } +func (m *Histogram) String() string { return proto.CompactTextString(m) } +func (*Histogram) ProtoMessage() {} +func (*Histogram) Descriptor() ([]byte, []int) { + return fileDescriptor_166e07899dab7c14, []int{3} +} +func (m *Histogram) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Histogram) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Histogram.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Histogram) XXX_Merge(src proto.Message) { + xxx_messageInfo_Histogram.Merge(m, src) +} +func (m *Histogram) XXX_Size() int { + return m.Size() +} +func (m *Histogram) XXX_DiscardUnknown() { + xxx_messageInfo_Histogram.DiscardUnknown(m) +} + +var xxx_messageInfo_Histogram proto.InternalMessageInfo + +type isHistogram_Count interface { + isHistogram_Count() + MarshalTo([]byte) (int, error) + Size() int +} +type isHistogram_ZeroCount interface { + isHistogram_ZeroCount() + MarshalTo([]byte) (int, error) + Size() int +} + +type Histogram_CountInt struct { + CountInt uint64 `protobuf:"varint,1,opt,name=count_int,json=countInt,proto3,oneof" json:"count_int,omitempty"` +} +type Histogram_CountFloat struct { + CountFloat float64 `protobuf:"fixed64,2,opt,name=count_float,json=countFloat,proto3,oneof" json:"count_float,omitempty"` +} +type Histogram_ZeroCountInt struct { + ZeroCountInt uint64 `protobuf:"varint,6,opt,name=zero_count_int,json=zeroCountInt,proto3,oneof" json:"zero_count_int,omitempty"` +} +type Histogram_ZeroCountFloat struct { + ZeroCountFloat float64 `protobuf:"fixed64,7,opt,name=zero_count_float,json=zeroCountFloat,proto3,oneof" json:"zero_count_float,omitempty"` +} + +func (*Histogram_CountInt) isHistogram_Count() {} +func (*Histogram_CountFloat) isHistogram_Count() {} +func (*Histogram_ZeroCountInt) isHistogram_ZeroCount() {} +func (*Histogram_ZeroCountFloat) isHistogram_ZeroCount() {} + +func (m *Histogram) GetCount() isHistogram_Count { + if m != nil { + return m.Count + } + return nil +} +func (m *Histogram) GetZeroCount() isHistogram_ZeroCount { + if m != nil { + return m.ZeroCount + } + return nil +} + +func (m *Histogram) GetCountInt() uint64 { + if x, ok := m.GetCount().(*Histogram_CountInt); ok { + return x.CountInt + } + return 0 +} + +func (m *Histogram) GetCountFloat() float64 { + if x, ok := m.GetCount().(*Histogram_CountFloat); ok { + return x.CountFloat + } + return 0 +} + +func (m *Histogram) GetSum() float64 { + if m != nil { + return m.Sum + } + return 0 +} + +func (m *Histogram) GetSchema() int32 { + if m != nil { + return m.Schema + } + return 0 +} + +func (m *Histogram) GetZeroThreshold() float64 { + if m != nil { + return m.ZeroThreshold + } + return 0 +} + +func (m *Histogram) GetZeroCountInt() uint64 { + if x, ok := m.GetZeroCount().(*Histogram_ZeroCountInt); ok { + return x.ZeroCountInt + } + return 0 +} + +func (m *Histogram) GetZeroCountFloat() float64 { + if x, ok := m.GetZeroCount().(*Histogram_ZeroCountFloat); ok { + return x.ZeroCountFloat + } + return 0 +} + +func (m *Histogram) GetNegativeSpans() []*BucketSpan { + if m != nil { + return m.NegativeSpans + } + return nil +} + +func (m *Histogram) GetNegativeDeltas() []int64 { + if m != nil { + return m.NegativeDeltas + } + return nil +} + +func (m *Histogram) GetNegativeCounts() []float64 { + if m != nil { + return m.NegativeCounts + } + return nil +} + +func (m *Histogram) GetPositiveSpans() []*BucketSpan { + if m != nil { + return m.PositiveSpans + } + return nil +} + +func (m *Histogram) GetPositiveDeltas() []int64 { + if m != nil { + return m.PositiveDeltas + } + return nil +} + +func (m *Histogram) GetPositiveCounts() []float64 { + if m != nil { + return m.PositiveCounts + } + return nil +} + +func (m *Histogram) GetResetHint() Histogram_ResetHint { + if m != nil { + return m.ResetHint + } + return Histogram_UNKNOWN +} + +func (m *Histogram) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Histogram) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Histogram_CountInt)(nil), + (*Histogram_CountFloat)(nil), + (*Histogram_ZeroCountInt)(nil), + (*Histogram_ZeroCountFloat)(nil), + } +} + +// A BucketSpan defines a number of consecutive buckets with their +// offset. Logically, it would be more straightforward to include the +// bucket counts in the Span. However, the protobuf representation is +// more compact in the way the data is structured here (with all the +// buckets in a single array separate from the Spans). +type BucketSpan struct { + Offset int32 `protobuf:"zigzag32,1,opt,name=offset,proto3" json:"offset,omitempty"` + Length uint32 `protobuf:"varint,2,opt,name=length,proto3" json:"length,omitempty"` +} + +func (m *BucketSpan) Reset() { *m = BucketSpan{} } +func (m *BucketSpan) String() string { return proto.CompactTextString(m) } +func (*BucketSpan) ProtoMessage() {} +func (*BucketSpan) Descriptor() ([]byte, []int) { + return fileDescriptor_166e07899dab7c14, []int{4} +} +func (m *BucketSpan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BucketSpan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_BucketSpan.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *BucketSpan) XXX_Merge(src proto.Message) { + xxx_messageInfo_BucketSpan.Merge(m, src) +} +func (m *BucketSpan) XXX_Size() int { + return m.Size() +} +func (m *BucketSpan) XXX_DiscardUnknown() { + xxx_messageInfo_BucketSpan.DiscardUnknown(m) +} + +var xxx_messageInfo_BucketSpan proto.InternalMessageInfo + +func (m *BucketSpan) GetOffset() int32 { + if m != nil { + return m.Offset + } + return 0 +} + +func (m *BucketSpan) GetLength() uint32 { + if m != nil { + return m.Length + } + return 0 +} + // TimeSeries represents samples and labels for a single time series. type TimeSeries struct { // Labels have to be sorted by label names and without duplicated label names. // TODO(bwplotka): Don't use zero copy ZLabels, see https://github.com/thanos-io/thanos/pull/3279 for details. - 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"` - Samples []Sample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples"` - Exemplars []Exemplar `protobuf:"bytes,3,rep,name=exemplars,proto3" json:"exemplars"` + 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"` + Samples []Sample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples"` + Exemplars []Exemplar `protobuf:"bytes,3,rep,name=exemplars,proto3" json:"exemplars"` + Histograms []Histogram `protobuf:"bytes,4,rep,name=histograms,proto3" json:"histograms"` } func (m *TimeSeries) Reset() { *m = TimeSeries{} } func (m *TimeSeries) String() string { return proto.CompactTextString(m) } func (*TimeSeries) ProtoMessage() {} func (*TimeSeries) Descriptor() ([]byte, []int) { - return fileDescriptor_166e07899dab7c14, []int{3} + return fileDescriptor_166e07899dab7c14, []int{5} } func (m *TimeSeries) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -361,6 +690,13 @@ func (m *TimeSeries) GetExemplars() []Exemplar { return nil } +func (m *TimeSeries) GetHistograms() []Histogram { + if m != nil { + return m.Histograms + } + return nil +} + // Matcher specifies a rule, which can match or set of labels or not. type LabelMatcher struct { Type LabelMatcher_Type `protobuf:"varint,1,opt,name=type,proto3,enum=prometheus_copy.LabelMatcher_Type" json:"type,omitempty"` @@ -372,7 +708,7 @@ func (m *LabelMatcher) Reset() { *m = LabelMatcher{} } func (m *LabelMatcher) String() string { return proto.CompactTextString(m) } func (*LabelMatcher) ProtoMessage() {} func (*LabelMatcher) Descriptor() ([]byte, []int) { - return fileDescriptor_166e07899dab7c14, []int{4} + return fileDescriptor_166e07899dab7c14, []int{6} } func (m *LabelMatcher) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -436,7 +772,7 @@ func (m *ReadHints) Reset() { *m = ReadHints{} } func (m *ReadHints) String() string { return proto.CompactTextString(m) } func (*ReadHints) ProtoMessage() {} func (*ReadHints) Descriptor() ([]byte, []int) { - return fileDescriptor_166e07899dab7c14, []int{5} + return fileDescriptor_166e07899dab7c14, []int{7} } func (m *ReadHints) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -527,7 +863,7 @@ func (m *Chunk) Reset() { *m = Chunk{} } func (m *Chunk) String() string { return proto.CompactTextString(m) } func (*Chunk) ProtoMessage() {} func (*Chunk) Descriptor() ([]byte, []int) { - return fileDescriptor_166e07899dab7c14, []int{6} + return fileDescriptor_166e07899dab7c14, []int{8} } func (m *Chunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -596,7 +932,7 @@ func (m *ChunkedSeries) Reset() { *m = ChunkedSeries{} } func (m *ChunkedSeries) String() string { return proto.CompactTextString(m) } func (*ChunkedSeries) ProtoMessage() {} func (*ChunkedSeries) Descriptor() ([]byte, []int) { - return fileDescriptor_166e07899dab7c14, []int{7} + return fileDescriptor_166e07899dab7c14, []int{9} } func (m *ChunkedSeries) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -634,11 +970,14 @@ func (m *ChunkedSeries) GetChunks() []Chunk { func init() { proto.RegisterEnum("prometheus_copy.MetricMetadata_MetricType", MetricMetadata_MetricType_name, MetricMetadata_MetricType_value) + proto.RegisterEnum("prometheus_copy.Histogram_ResetHint", Histogram_ResetHint_name, Histogram_ResetHint_value) proto.RegisterEnum("prometheus_copy.LabelMatcher_Type", LabelMatcher_Type_name, LabelMatcher_Type_value) proto.RegisterEnum("prometheus_copy.Chunk_Encoding", Chunk_Encoding_name, Chunk_Encoding_value) proto.RegisterType((*MetricMetadata)(nil), "prometheus_copy.MetricMetadata") proto.RegisterType((*Sample)(nil), "prometheus_copy.Sample") proto.RegisterType((*Exemplar)(nil), "prometheus_copy.Exemplar") + proto.RegisterType((*Histogram)(nil), "prometheus_copy.Histogram") + proto.RegisterType((*BucketSpan)(nil), "prometheus_copy.BucketSpan") proto.RegisterType((*TimeSeries)(nil), "prometheus_copy.TimeSeries") proto.RegisterType((*LabelMatcher)(nil), "prometheus_copy.LabelMatcher") proto.RegisterType((*ReadHints)(nil), "prometheus_copy.ReadHints") @@ -649,57 +988,79 @@ func init() { func init() { proto.RegisterFile("store/storepb/prompb/types.proto", fileDescriptor_166e07899dab7c14) } var fileDescriptor_166e07899dab7c14 = []byte{ - // 796 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x55, 0x4f, 0x8f, 0xdb, 0x44, - 0x14, 0xcf, 0xd8, 0x8e, 0x1d, 0xbf, 0xfd, 0x83, 0x35, 0x2a, 0xd4, 0xbb, 0x42, 0x59, 0xcb, 0xa7, - 0x08, 0x81, 0x23, 0xb5, 0x15, 0x5c, 0x0a, 0xd2, 0x6e, 0xe5, 0x6e, 0x2b, 0x70, 0xa2, 0x4e, 0xb2, - 0x02, 0x7a, 0x89, 0x26, 0xce, 0xd4, 0xb1, 0x1a, 0xff, 0x91, 0x67, 0x82, 0x36, 0xdf, 0x82, 0x33, - 0x37, 0xc4, 0x8d, 0x1b, 0x7c, 0x8a, 0x1e, 0x7b, 0x44, 0x1c, 0x2a, 0xb4, 0x7b, 0xe2, 0x5b, 0xa0, - 0x19, 0x3b, 0xf5, 0xa6, 0x4b, 0xaf, 0xbd, 0xac, 0xde, 0xfb, 0xfd, 0xde, 0x3f, 0xbf, 0xf7, 0xdb, - 0x09, 0x78, 0x5c, 0x14, 0x15, 0x1b, 0xaa, 0xbf, 0xe5, 0x7c, 0x58, 0x56, 0x45, 0x56, 0xce, 0x87, - 0x62, 0x53, 0x32, 0x1e, 0x94, 0x55, 0x21, 0x0a, 0xfc, 0x91, 0xc4, 0x98, 0x58, 0xb2, 0x35, 0x9f, - 0xc5, 0x45, 0xb9, 0x39, 0xbe, 0x93, 0x14, 0x49, 0xa1, 0xb8, 0xa1, 0xb4, 0xea, 0xb0, 0xe3, 0xa3, - 0xba, 0xd0, 0x8a, 0xce, 0xd9, 0x6a, 0xb7, 0x82, 0xff, 0xab, 0x06, 0x87, 0x11, 0x13, 0x55, 0x1a, - 0x47, 0x4c, 0xd0, 0x05, 0x15, 0x14, 0x7f, 0x03, 0x86, 0x8c, 0x70, 0x91, 0x87, 0x06, 0x87, 0xf7, - 0x3e, 0x0b, 0xde, 0xe9, 0x11, 0xec, 0x86, 0x37, 0xee, 0x74, 0x53, 0x32, 0xa2, 0xf2, 0xf0, 0xe7, - 0x80, 0x33, 0x85, 0xcd, 0x5e, 0xd0, 0x2c, 0x5d, 0x6d, 0x66, 0x39, 0xcd, 0x98, 0xab, 0x79, 0x68, - 0x60, 0x13, 0xa7, 0x66, 0x1e, 0x2b, 0x62, 0x44, 0x33, 0x86, 0x31, 0x18, 0x4b, 0xb6, 0x2a, 0x5d, - 0x43, 0xf1, 0xca, 0x96, 0xd8, 0x3a, 0x4f, 0x85, 0xdb, 0xad, 0x31, 0x69, 0xfb, 0x1b, 0x80, 0xb6, - 0x13, 0xde, 0x03, 0xeb, 0x62, 0xf4, 0xed, 0x68, 0xfc, 0xfd, 0xc8, 0xe9, 0x48, 0xe7, 0xd1, 0xf8, - 0x62, 0x34, 0x0d, 0x89, 0x83, 0xb0, 0x0d, 0xdd, 0xf3, 0xd3, 0x8b, 0xf3, 0xd0, 0xd1, 0xf0, 0x01, - 0xd8, 0x4f, 0x9e, 0x4e, 0xa6, 0xe3, 0x73, 0x72, 0x1a, 0x39, 0x3a, 0xc6, 0x70, 0xa8, 0x98, 0x16, - 0x33, 0x64, 0xea, 0xe4, 0x22, 0x8a, 0x4e, 0xc9, 0x8f, 0x4e, 0x17, 0xf7, 0xc0, 0x78, 0x3a, 0x7a, - 0x3c, 0x76, 0x4c, 0xbc, 0x0f, 0xbd, 0xc9, 0xf4, 0x74, 0x1a, 0x4e, 0xc2, 0xa9, 0x63, 0xf9, 0x0f, - 0xc1, 0x9c, 0xd0, 0xac, 0x5c, 0x31, 0x7c, 0x07, 0xba, 0x3f, 0xd1, 0xd5, 0xba, 0xde, 0x0d, 0x22, - 0xb5, 0x83, 0x3f, 0x05, 0x5b, 0xa4, 0x19, 0xe3, 0x82, 0x66, 0xa5, 0xfa, 0x4e, 0x9d, 0xb4, 0x80, - 0xff, 0x1b, 0x82, 0x5e, 0x78, 0xc9, 0xb2, 0x72, 0x45, 0x2b, 0x1c, 0x83, 0xa9, 0xae, 0xc0, 0x5d, - 0xe4, 0xe9, 0x83, 0xbd, 0x7b, 0x07, 0x81, 0x58, 0xd2, 0xbc, 0xe0, 0xc1, 0x77, 0x12, 0x3d, 0x7b, - 0xf8, 0xea, 0xcd, 0x49, 0xe7, 0xef, 0x37, 0x27, 0x0f, 0x92, 0x54, 0x2c, 0xd7, 0xf3, 0x20, 0x2e, - 0xb2, 0x61, 0x1d, 0xf0, 0x45, 0x5a, 0x34, 0xd6, 0xb0, 0x7c, 0x99, 0x0c, 0x77, 0x0e, 0x1a, 0x3c, - 0x57, 0xd9, 0xa4, 0x29, 0xdd, 0x4e, 0xa9, 0xbd, 0x77, 0x4a, 0xfd, 0xdd, 0x29, 0xff, 0x45, 0x00, - 0xd3, 0x34, 0x63, 0x13, 0x56, 0xa5, 0x8c, 0x7f, 0x98, 0x39, 0xbf, 0x02, 0x8b, 0xab, 0xbd, 0x72, - 0x57, 0x53, 0x5d, 0xee, 0xde, 0xd2, 0x5a, 0xbd, 0xf7, 0x33, 0x43, 0xf6, 0x23, 0xdb, 0x68, 0xfc, - 0x35, 0xd8, 0xac, 0xd9, 0x28, 0x77, 0x75, 0x95, 0x7a, 0x74, 0x2b, 0x75, 0xbb, 0xf3, 0x26, 0xb9, - 0xcd, 0xf0, 0x7f, 0x41, 0xb0, 0xaf, 0x26, 0x89, 0xa8, 0x88, 0x97, 0xac, 0xc2, 0x5f, 0xee, 0x28, - 0xde, 0xbf, 0x55, 0xea, 0x66, 0x70, 0x70, 0x43, 0xe9, 0x18, 0x8c, 0x1b, 0xda, 0x56, 0x76, 0xbb, - 0x7c, 0x5d, 0x81, 0xb5, 0xe3, 0x0f, 0xc0, 0x50, 0xba, 0x35, 0x41, 0x0b, 0x9f, 0x39, 0x1d, 0x6c, - 0x81, 0x3e, 0x0a, 0x9f, 0x39, 0x48, 0x02, 0x44, 0x6a, 0x55, 0x02, 0x24, 0x74, 0x74, 0xff, 0x0f, - 0x04, 0x36, 0x61, 0x74, 0xf1, 0x24, 0xcd, 0x05, 0xc7, 0x77, 0xc1, 0xe2, 0x82, 0x95, 0xb3, 0x8c, - 0xab, 0xe1, 0x74, 0x62, 0x4a, 0x37, 0xe2, 0xb2, 0xf5, 0x8b, 0x75, 0x1e, 0x6f, 0x5b, 0x4b, 0x1b, - 0x1f, 0x41, 0x8f, 0x0b, 0x5a, 0x09, 0x19, 0x5d, 0x1f, 0xd8, 0x52, 0x7e, 0xc4, 0xf1, 0xc7, 0x60, - 0xb2, 0x7c, 0x21, 0x09, 0x43, 0x11, 0x5d, 0x96, 0x2f, 0x22, 0x8e, 0x8f, 0xa1, 0x97, 0x54, 0xc5, - 0xba, 0x4c, 0xf3, 0xc4, 0xed, 0x7a, 0xfa, 0xc0, 0x26, 0x6f, 0x7d, 0x7c, 0x08, 0xda, 0x7c, 0xe3, - 0x9a, 0x1e, 0x1a, 0xf4, 0x88, 0x36, 0xdf, 0xc8, 0xea, 0x15, 0xcd, 0x13, 0x26, 0x8b, 0x58, 0x75, - 0x75, 0xe5, 0x47, 0xdc, 0xff, 0x13, 0x41, 0xf7, 0xd1, 0x72, 0x9d, 0xbf, 0xc4, 0x7d, 0xd8, 0xcb, - 0xd2, 0x7c, 0x26, 0x75, 0xd5, 0xce, 0x6c, 0x67, 0x69, 0x2e, 0xb5, 0x15, 0x71, 0xc5, 0xd3, 0xcb, - 0xb7, 0x7c, 0xf3, 0xcf, 0x92, 0xd1, 0xcb, 0x86, 0xbf, 0xdf, 0x5c, 0x42, 0x57, 0x97, 0x38, 0xb9, - 0x75, 0x09, 0xd5, 0x25, 0x08, 0xf3, 0xb8, 0x58, 0xa4, 0x79, 0xd2, 0x9e, 0x41, 0xbe, 0x44, 0xea, - 0xd3, 0xf6, 0x89, 0xb2, 0x7d, 0x0f, 0x7a, 0xdb, 0xa8, 0xdd, 0xc7, 0xc2, 0x02, 0xfd, 0x87, 0x31, - 0x71, 0x90, 0xff, 0x3b, 0x82, 0x03, 0x55, 0x8e, 0x2d, 0x3e, 0xa4, 0xe8, 0x1f, 0x80, 0x19, 0xcb, - 0xae, 0x5b, 0xcd, 0x7f, 0xf2, 0xff, 0xdf, 0xd8, 0xa8, 0xb6, 0x89, 0x3d, 0xf3, 0x5e, 0x5d, 0xf5, - 0xd1, 0xeb, 0xab, 0x3e, 0xfa, 0xe7, 0xaa, 0x8f, 0x7e, 0xbe, 0xee, 0x77, 0x5e, 0x5f, 0xf7, 0x3b, - 0x7f, 0x5d, 0xf7, 0x3b, 0xcf, 0xcd, 0xfa, 0x67, 0x61, 0x6e, 0xaa, 0xf7, 0xfc, 0xfe, 0x7f, 0x01, - 0x00, 0x00, 0xff, 0xff, 0x98, 0xc8, 0x2e, 0xfd, 0x35, 0x06, 0x00, 0x00, + // 1139 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0x4b, 0x8f, 0x1a, 0xc7, + 0x13, 0x67, 0x66, 0x60, 0x60, 0x6a, 0x01, 0x8f, 0x5b, 0xfe, 0xdb, 0xe3, 0xfd, 0x27, 0x2c, 0x19, + 0xe5, 0x81, 0xac, 0x04, 0x24, 0x7b, 0x95, 0x5c, 0x36, 0x51, 0x96, 0x0d, 0xfb, 0x50, 0x02, 0xc8, + 0x0d, 0xab, 0xc4, 0xbe, 0xa0, 0x06, 0x7a, 0x99, 0xd1, 0x32, 0x0f, 0x4d, 0x37, 0xd6, 0x92, 0x4f, + 0x91, 0x73, 0x6e, 0x51, 0x6e, 0xc9, 0x29, 0x1f, 0x21, 0x37, 0x9f, 0x22, 0x1f, 0xa3, 0x1c, 0xac, + 0x68, 0xf7, 0x8b, 0x44, 0xdd, 0x33, 0xc3, 0xc0, 0x12, 0x4b, 0x39, 0xf9, 0x82, 0xaa, 0x7e, 0xf5, + 0xfa, 0x4d, 0x57, 0x75, 0x35, 0x50, 0x67, 0x3c, 0x88, 0x68, 0x4b, 0xfe, 0x86, 0xe3, 0x56, 0x18, + 0x05, 0x5e, 0x38, 0x6e, 0xf1, 0x65, 0x48, 0x59, 0x33, 0x8c, 0x02, 0x1e, 0xa0, 0x3b, 0x02, 0xa3, + 0xdc, 0xa1, 0x0b, 0x36, 0x9a, 0x04, 0xe1, 0x72, 0xf7, 0xde, 0x2c, 0x98, 0x05, 0xd2, 0xd6, 0x12, + 0x52, 0xec, 0xb6, 0xfb, 0x30, 0x4e, 0x34, 0x27, 0x63, 0x3a, 0xdf, 0xcc, 0x60, 0xff, 0xa4, 0x42, + 0xb5, 0x4b, 0x79, 0xe4, 0x4e, 0xba, 0x94, 0x93, 0x29, 0xe1, 0x04, 0x7d, 0x01, 0x79, 0xe1, 0x61, + 0x29, 0x75, 0xa5, 0x51, 0x7d, 0xfc, 0xa8, 0x79, 0xab, 0x46, 0x73, 0xd3, 0x3d, 0x51, 0x87, 0xcb, + 0x90, 0x62, 0x19, 0x87, 0x3e, 0x06, 0xe4, 0x49, 0x6c, 0x74, 0x41, 0x3c, 0x77, 0xbe, 0x1c, 0xf9, + 0xc4, 0xa3, 0x96, 0x5a, 0x57, 0x1a, 0x06, 0x36, 0x63, 0xcb, 0xb1, 0x34, 0xf4, 0x88, 0x47, 0x11, + 0x82, 0xbc, 0x43, 0xe7, 0xa1, 0x95, 0x97, 0x76, 0x29, 0x0b, 0x6c, 0xe1, 0xbb, 0xdc, 0x2a, 0xc4, + 0x98, 0x90, 0xed, 0x25, 0x40, 0x56, 0x09, 0xed, 0x40, 0xf1, 0xbc, 0xf7, 0x75, 0xaf, 0xff, 0x6d, + 0xcf, 0xcc, 0x09, 0xe5, 0xa8, 0x7f, 0xde, 0x1b, 0x76, 0xb0, 0xa9, 0x20, 0x03, 0x0a, 0x27, 0x87, + 0xe7, 0x27, 0x1d, 0x53, 0x45, 0x15, 0x30, 0x4e, 0xcf, 0x06, 0xc3, 0xfe, 0x09, 0x3e, 0xec, 0x9a, + 0x1a, 0x42, 0x50, 0x95, 0x96, 0x0c, 0xcb, 0x8b, 0xd0, 0xc1, 0x79, 0xb7, 0x7b, 0x88, 0x9f, 0x99, + 0x05, 0x54, 0x82, 0xfc, 0x59, 0xef, 0xb8, 0x6f, 0xea, 0xa8, 0x0c, 0xa5, 0xc1, 0xf0, 0x70, 0xd8, + 0x19, 0x74, 0x86, 0x66, 0xd1, 0x3e, 0x00, 0x7d, 0x40, 0xbc, 0x70, 0x4e, 0xd1, 0x3d, 0x28, 0xbc, + 0x20, 0xf3, 0x45, 0x7c, 0x36, 0x0a, 0x8e, 0x15, 0xf4, 0x0e, 0x18, 0xdc, 0xf5, 0x28, 0xe3, 0xc4, + 0x0b, 0xe5, 0x77, 0x6a, 0x38, 0x03, 0xec, 0x9f, 0x15, 0x28, 0x75, 0xae, 0xa8, 0x17, 0xce, 0x49, + 0x84, 0x26, 0xa0, 0xcb, 0x2e, 0x30, 0x4b, 0xa9, 0x6b, 0x8d, 0x9d, 0xc7, 0x95, 0x26, 0x77, 0x88, + 0x1f, 0xb0, 0xe6, 0x37, 0x02, 0x6d, 0x1f, 0xbc, 0x7c, 0xbd, 0x97, 0xfb, 0xeb, 0xf5, 0xde, 0xfe, + 0xcc, 0xe5, 0xce, 0x62, 0xdc, 0x9c, 0x04, 0x5e, 0x2b, 0x76, 0xf8, 0xc4, 0x0d, 0x12, 0xa9, 0x15, + 0x5e, 0xce, 0x5a, 0x1b, 0x0d, 0x6d, 0x3e, 0x97, 0xd1, 0x38, 0x49, 0x9d, 0xb1, 0x54, 0xdf, 0xc8, + 0x52, 0xbb, 0xcd, 0xf2, 0x8f, 0x02, 0x18, 0xa7, 0x2e, 0xe3, 0xc1, 0x2c, 0x22, 0x1e, 0x7a, 0x17, + 0x8c, 0x49, 0xb0, 0xf0, 0xf9, 0xc8, 0xf5, 0xb9, 0xfc, 0xd6, 0xfc, 0x69, 0x0e, 0x97, 0x24, 0x74, + 0xe6, 0x73, 0xf4, 0x1e, 0xec, 0xc4, 0xe6, 0x8b, 0x79, 0x40, 0x78, 0x5c, 0xe6, 0x34, 0x87, 0x41, + 0x82, 0xc7, 0x02, 0x43, 0x26, 0x68, 0x6c, 0xe1, 0xc9, 0x3a, 0x0a, 0x16, 0x22, 0xba, 0x0f, 0x3a, + 0x9b, 0x38, 0xd4, 0x23, 0xb2, 0xd5, 0x77, 0x71, 0xa2, 0xa1, 0x0f, 0xa0, 0xfa, 0x3d, 0x8d, 0x82, + 0x11, 0x77, 0x22, 0xca, 0x9c, 0x60, 0x3e, 0x95, 0x6d, 0x57, 0x70, 0x45, 0xa0, 0xc3, 0x14, 0x44, + 0x1f, 0x26, 0x6e, 0x19, 0x2f, 0x5d, 0xf2, 0x52, 0x70, 0x59, 0xe0, 0x47, 0x29, 0xb7, 0x47, 0x60, + 0xae, 0xf9, 0xc5, 0x04, 0x8b, 0x92, 0xa0, 0x82, 0xab, 0x2b, 0xcf, 0x98, 0x64, 0x1b, 0xaa, 0x3e, + 0x9d, 0x11, 0xee, 0xbe, 0xa0, 0x23, 0x16, 0x12, 0x9f, 0x59, 0x25, 0xd9, 0x95, 0xff, 0x6f, 0xcd, + 0x7c, 0x7b, 0x31, 0xb9, 0xa4, 0x7c, 0x10, 0x12, 0x1f, 0x57, 0xd2, 0x10, 0xa1, 0x31, 0xf4, 0x11, + 0xdc, 0x59, 0xe5, 0x98, 0xd2, 0x39, 0x27, 0xcc, 0x32, 0xea, 0x5a, 0x03, 0xe1, 0x55, 0xea, 0xaf, + 0x24, 0xba, 0xe1, 0x28, 0xc9, 0x31, 0x0b, 0xea, 0x5a, 0x43, 0xc9, 0x1c, 0x25, 0x33, 0x26, 0x58, + 0x85, 0x01, 0x73, 0xd7, 0x58, 0xed, 0xfc, 0x07, 0x56, 0x69, 0xc8, 0x8a, 0xd5, 0x2a, 0x47, 0xc2, + 0xaa, 0x1c, 0xb3, 0x4a, 0xe1, 0x8c, 0xd5, 0xca, 0x31, 0x61, 0x55, 0x89, 0x59, 0xa5, 0x70, 0xc2, + 0xea, 0x08, 0x20, 0xa2, 0x8c, 0xf2, 0x91, 0x23, 0xce, 0xbe, 0x2a, 0x77, 0xc3, 0xfb, 0x5b, 0x8c, + 0x56, 0x23, 0xd4, 0xc4, 0xc2, 0xf9, 0xd4, 0xf5, 0x39, 0x36, 0xa2, 0x54, 0xdc, 0x9c, 0xc1, 0x3b, + 0xb7, 0x67, 0x70, 0x1f, 0x8c, 0x55, 0xd4, 0xe6, 0x0d, 0x2f, 0x82, 0xf6, 0xac, 0x33, 0x30, 0x15, + 0xa4, 0x83, 0xda, 0xeb, 0x9b, 0x6a, 0x76, 0xcb, 0xb5, 0x76, 0x11, 0x0a, 0x92, 0x78, 0xbb, 0x0c, + 0x90, 0x75, 0xde, 0x3e, 0x00, 0xc8, 0x8e, 0x47, 0x0c, 0x5f, 0x70, 0x71, 0xc1, 0x68, 0x3c, 0xcd, + 0x77, 0x71, 0xa2, 0x09, 0x7c, 0x4e, 0xfd, 0x19, 0x77, 0xe4, 0x10, 0x57, 0x70, 0xa2, 0xd9, 0xbf, + 0xaa, 0x00, 0x43, 0xd7, 0xa3, 0x03, 0x1a, 0xb9, 0x94, 0xbd, 0x9d, 0x6b, 0xfb, 0x19, 0x14, 0x99, + 0x5c, 0x33, 0xcc, 0x52, 0x65, 0x95, 0x07, 0x5b, 0xc7, 0x1b, 0xaf, 0xa1, 0x76, 0x5e, 0xd4, 0xc3, + 0xa9, 0x37, 0xfa, 0x1c, 0x0c, 0x9a, 0x2c, 0x18, 0x66, 0x69, 0x32, 0xf4, 0xe1, 0x56, 0x68, 0xba, + 0x82, 0x92, 0xe0, 0x2c, 0x02, 0x7d, 0x09, 0xe0, 0xa4, 0x6d, 0x63, 0x56, 0x5e, 0xc6, 0xef, 0xbe, + 0xb9, 0xb3, 0x49, 0x82, 0xb5, 0x18, 0xfb, 0x47, 0x05, 0xca, 0xf2, 0x5b, 0xba, 0x84, 0x4f, 0x1c, + 0x1a, 0xa1, 0x4f, 0x37, 0x9e, 0x10, 0x7b, 0x2b, 0xd9, 0xba, 0x73, 0x73, 0xed, 0xe9, 0x40, 0x90, + 0x5f, 0x7b, 0x2c, 0xa4, 0x9c, 0x6d, 0x33, 0x4d, 0x82, 0xb1, 0x62, 0x37, 0x20, 0x2f, 0x1f, 0x02, + 0x1d, 0xd4, 0xce, 0xd3, 0x78, 0x42, 0x7a, 0x9d, 0xa7, 0xf1, 0x84, 0x60, 0xb1, 0xfc, 0x05, 0x80, + 0x3b, 0xa6, 0x66, 0xff, 0xa6, 0x88, 0xb1, 0x22, 0x53, 0x31, 0x55, 0x0c, 0x3d, 0x80, 0x22, 0xe3, + 0x34, 0x1c, 0x79, 0x4c, 0x92, 0xd3, 0xb0, 0x2e, 0xd4, 0x2e, 0x13, 0xa5, 0x2f, 0x16, 0xfe, 0x24, + 0x2d, 0x2d, 0x64, 0xf4, 0x10, 0x4a, 0x8c, 0x93, 0x88, 0x0b, 0xef, 0x78, 0x63, 0x16, 0xa5, 0xde, + 0x65, 0xe8, 0x7f, 0xa0, 0x53, 0x7f, 0x3a, 0x92, 0x07, 0x26, 0x0c, 0x05, 0xea, 0x4f, 0xbb, 0x0c, + 0xed, 0x42, 0x69, 0x16, 0x05, 0x8b, 0xd0, 0xf5, 0x67, 0x56, 0xa1, 0xae, 0x35, 0x0c, 0xbc, 0xd2, + 0x51, 0x15, 0xd4, 0xf1, 0x52, 0x6e, 0xad, 0x12, 0x56, 0xc7, 0x4b, 0x91, 0x3d, 0x22, 0xfe, 0x8c, + 0x8a, 0x24, 0xc5, 0x38, 0xbb, 0xd4, 0xbb, 0xcc, 0xfe, 0x5d, 0x81, 0xc2, 0x91, 0xb3, 0xf0, 0x2f, + 0x51, 0x0d, 0x76, 0x3c, 0xd7, 0x1f, 0x89, 0x4b, 0x92, 0x71, 0x36, 0x3c, 0xd7, 0x17, 0xd3, 0xd9, + 0x65, 0xd2, 0x4e, 0xae, 0x56, 0xf6, 0xe4, 0xf5, 0xf1, 0xc8, 0x55, 0x62, 0x7f, 0x92, 0x74, 0x42, + 0x93, 0x9d, 0xd8, 0xdb, 0xea, 0x84, 0xac, 0xd2, 0xec, 0xf8, 0x93, 0x60, 0xea, 0xfa, 0xb3, 0xac, + 0x0d, 0xe2, 0x69, 0x97, 0x9f, 0x56, 0xc6, 0x52, 0xb6, 0x5b, 0x50, 0x4a, 0xbd, 0xb6, 0xee, 0xe6, + 0x77, 0x7d, 0xf1, 0xf2, 0x6e, 0x3c, 0xb7, 0xaa, 0xfd, 0x8b, 0x02, 0x15, 0x99, 0x9d, 0x4e, 0xdf, + 0xe6, 0x2d, 0xda, 0x07, 0x7d, 0x22, 0xaa, 0xa6, 0x97, 0xe8, 0xfe, 0xbf, 0x7f, 0x72, 0x32, 0xc5, + 0x89, 0x6f, 0xbb, 0xfe, 0xf2, 0xba, 0xa6, 0xbc, 0xba, 0xae, 0x29, 0x7f, 0x5f, 0xd7, 0x94, 0x1f, + 0x6e, 0x6a, 0xb9, 0x57, 0x37, 0xb5, 0xdc, 0x9f, 0x37, 0xb5, 0xdc, 0x73, 0x3d, 0xfe, 0xdb, 0x35, + 0xd6, 0xe5, 0xff, 0xa5, 0x27, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x0e, 0x67, 0xfa, 0x2f, 0x95, + 0x09, 0x00, 0x00, } func (m *MetricMetadata) Marshal() (dAtA []byte, err error) { @@ -833,7 +1194,7 @@ func (m *Exemplar) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *TimeSeries) Marshal() (dAtA []byte, err error) { +func (m *Histogram) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -843,20 +1204,59 @@ func (m *TimeSeries) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *TimeSeries) MarshalTo(dAtA []byte) (int, error) { +func (m *Histogram) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *TimeSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *Histogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if len(m.Exemplars) > 0 { - for iNdEx := len(m.Exemplars) - 1; iNdEx >= 0; iNdEx-- { + if m.Timestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x78 + } + if m.ResetHint != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.ResetHint)) + i-- + dAtA[i] = 0x70 + } + if len(m.PositiveCounts) > 0 { + for iNdEx := len(m.PositiveCounts) - 1; iNdEx >= 0; iNdEx-- { + f1 := math.Float64bits(float64(m.PositiveCounts[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f1)) + } + i = encodeVarintTypes(dAtA, i, uint64(len(m.PositiveCounts)*8)) + i-- + dAtA[i] = 0x6a + } + if len(m.PositiveDeltas) > 0 { + var j2 int + dAtA4 := make([]byte, len(m.PositiveDeltas)*10) + for _, num := range m.PositiveDeltas { + x3 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x3 >= 1<<7 { + dAtA4[j2] = uint8(uint64(x3)&0x7f | 0x80) + j2++ + x3 >>= 7 + } + dAtA4[j2] = uint8(x3) + j2++ + } + i -= j2 + copy(dAtA[i:], dAtA4[:j2]) + i = encodeVarintTypes(dAtA, i, uint64(j2)) + i-- + dAtA[i] = 0x62 + } + if len(m.PositiveSpans) > 0 { + for iNdEx := len(m.PositiveSpans) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Exemplars[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.PositiveSpans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -864,13 +1264,42 @@ func (m *TimeSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintTypes(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x1a + dAtA[i] = 0x5a } } - if len(m.Samples) > 0 { - for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { + if len(m.NegativeCounts) > 0 { + for iNdEx := len(m.NegativeCounts) - 1; iNdEx >= 0; iNdEx-- { + f5 := math.Float64bits(float64(m.NegativeCounts[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f5)) + } + i = encodeVarintTypes(dAtA, i, uint64(len(m.NegativeCounts)*8)) + i-- + dAtA[i] = 0x52 + } + if len(m.NegativeDeltas) > 0 { + var j6 int + dAtA8 := make([]byte, len(m.NegativeDeltas)*10) + for _, num := range m.NegativeDeltas { + x7 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x7 >= 1<<7 { + dAtA8[j6] = uint8(uint64(x7)&0x7f | 0x80) + j6++ + x7 >>= 7 + } + dAtA8[j6] = uint8(x7) + j6++ + } + i -= j6 + copy(dAtA[i:], dAtA8[:j6]) + i = encodeVarintTypes(dAtA, i, uint64(j6)) + i-- + dAtA[i] = 0x4a + } + if len(m.NegativeSpans) > 0 { + for iNdEx := len(m.NegativeSpans) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.NegativeSpans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -878,27 +1307,210 @@ func (m *TimeSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintTypes(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x12 + dAtA[i] = 0x42 } } - if len(m.Labels) > 0 { - for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { - { - size := m.Labels[iNdEx].Size() - i -= size - if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - i = encodeVarintTypes(dAtA, i, uint64(size)) + if m.ZeroCount != nil { + { + size := m.ZeroCount.Size() + i -= size + if _, err := m.ZeroCount.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.ZeroThreshold != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.ZeroThreshold)))) + i-- + dAtA[i] = 0x29 + } + if m.Schema != 0 { + i = encodeVarintTypes(dAtA, i, uint64((uint32(m.Schema)<<1)^uint32((m.Schema>>31)))) + i-- + dAtA[i] = 0x20 + } + if m.Sum != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Sum)))) + i-- + dAtA[i] = 0x19 + } + if m.Count != nil { + { + size := m.Count.Size() + i -= size + if _, err := m.Count.MarshalTo(dAtA[i:]); err != nil { + return 0, err } - i-- - dAtA[i] = 0xa } } return len(dAtA) - i, nil } -func (m *LabelMatcher) Marshal() (dAtA []byte, err error) { +func (m *Histogram_CountInt) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_CountInt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintTypes(dAtA, i, uint64(m.CountInt)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} +func (m *Histogram_CountFloat) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_CountFloat) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.CountFloat)))) + i-- + dAtA[i] = 0x11 + return len(dAtA) - i, nil +} +func (m *Histogram_ZeroCountInt) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_ZeroCountInt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintTypes(dAtA, i, uint64(m.ZeroCountInt)) + i-- + dAtA[i] = 0x30 + return len(dAtA) - i, nil +} +func (m *Histogram_ZeroCountFloat) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_ZeroCountFloat) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.ZeroCountFloat)))) + i-- + dAtA[i] = 0x39 + return len(dAtA) - i, nil +} +func (m *BucketSpan) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BucketSpan) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BucketSpan) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Length != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Length)) + i-- + dAtA[i] = 0x10 + } + if m.Offset != 0 { + i = encodeVarintTypes(dAtA, i, uint64((uint32(m.Offset)<<1)^uint32((m.Offset>>31)))) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *TimeSeries) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TimeSeries) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TimeSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Histograms) > 0 { + for iNdEx := len(m.Histograms) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Histograms[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + } + if len(m.Exemplars) > 0 { + for iNdEx := len(m.Exemplars) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Exemplars[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.Samples) > 0 { + for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Labels[iNdEx].Size() + i -= size + if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *LabelMatcher) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1176,6 +1788,119 @@ func (m *Exemplar) Size() (n int) { return n } +func (m *Histogram) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Count != nil { + n += m.Count.Size() + } + if m.Sum != 0 { + n += 9 + } + if m.Schema != 0 { + n += 1 + sozTypes(uint64(m.Schema)) + } + if m.ZeroThreshold != 0 { + n += 9 + } + if m.ZeroCount != nil { + n += m.ZeroCount.Size() + } + if len(m.NegativeSpans) > 0 { + for _, e := range m.NegativeSpans { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.NegativeDeltas) > 0 { + l = 0 + for _, e := range m.NegativeDeltas { + l += sozTypes(uint64(e)) + } + n += 1 + sovTypes(uint64(l)) + l + } + if len(m.NegativeCounts) > 0 { + n += 1 + sovTypes(uint64(len(m.NegativeCounts)*8)) + len(m.NegativeCounts)*8 + } + if len(m.PositiveSpans) > 0 { + for _, e := range m.PositiveSpans { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.PositiveDeltas) > 0 { + l = 0 + for _, e := range m.PositiveDeltas { + l += sozTypes(uint64(e)) + } + n += 1 + sovTypes(uint64(l)) + l + } + if len(m.PositiveCounts) > 0 { + n += 1 + sovTypes(uint64(len(m.PositiveCounts)*8)) + len(m.PositiveCounts)*8 + } + if m.ResetHint != 0 { + n += 1 + sovTypes(uint64(m.ResetHint)) + } + if m.Timestamp != 0 { + n += 1 + sovTypes(uint64(m.Timestamp)) + } + return n +} + +func (m *Histogram_CountInt) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovTypes(uint64(m.CountInt)) + return n +} +func (m *Histogram_CountFloat) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 9 + return n +} +func (m *Histogram_ZeroCountInt) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovTypes(uint64(m.ZeroCountInt)) + return n +} +func (m *Histogram_ZeroCountFloat) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 9 + return n +} +func (m *BucketSpan) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Offset != 0 { + n += 1 + sozTypes(uint64(m.Offset)) + } + if m.Length != 0 { + n += 1 + sovTypes(uint64(m.Length)) + } + return n +} + func (m *TimeSeries) Size() (n int) { if m == nil { return 0 @@ -1200,6 +1925,12 @@ func (m *TimeSeries) Size() (n int) { n += 1 + l + sovTypes(uint64(l)) } } + if len(m.Histograms) > 0 { + for _, e := range m.Histograms { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } return n } @@ -1665,7 +2396,7 @@ func (m *Exemplar) Unmarshal(dAtA []byte) error { } return nil } -func (m *TimeSeries) Unmarshal(dAtA []byte) error { +func (m *Histogram) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1688,17 +2419,17 @@ func (m *TimeSeries) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TimeSeries: wiretype end group for non-group") + return fmt.Errorf("proto: Histogram: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TimeSeries: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Histogram: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CountInt", wireType) } - var msglen int + var v uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowTypes @@ -1708,31 +2439,39 @@ func (m *TimeSeries) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthTypes - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthTypes + m.Count = &Histogram_CountInt{v} + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field CountFloat", wireType) } - if postIndex > l { + var v uint64 + if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.Labels = append(m.Labels, github_com_thanos_io_thanos_pkg_store_labelpb.ZLabel{}) - if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Count = &Histogram_CountFloat{float64(math.Float64frombits(v))} + case 3: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Sum", wireType) } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF } - var msglen int + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Sum = float64(math.Float64frombits(v)) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) + } + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowTypes @@ -1742,31 +2481,29 @@ func (m *TimeSeries) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthTypes - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthTypes + v = int32((uint32(v) >> 1) ^ uint32(((v&1)<<31)>>31)) + m.Schema = v + case 5: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroThreshold", wireType) } - if postIndex > l { + var v uint64 + if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.Samples = append(m.Samples, Sample{}) - if err := m.Samples[len(m.Samples)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Exemplars", wireType) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.ZeroThreshold = float64(math.Float64frombits(v)) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroCountInt", wireType) } - var msglen int + var v uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowTypes @@ -1776,26 +2513,669 @@ func (m *TimeSeries) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthTypes - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthTypes + m.ZeroCount = &Histogram_ZeroCountInt{v} + case 7: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroCountFloat", wireType) } - if postIndex > l { + var v uint64 + if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.Exemplars = append(m.Exemplars, Exemplar{}) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.ZeroCount = &Histogram_ZeroCountFloat{float64(math.Float64frombits(v))} + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeSpans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NegativeSpans = append(m.NegativeSpans, &BucketSpan{}) + if err := m.NegativeSpans[len(m.NegativeSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.NegativeDeltas = append(m.NegativeDeltas, int64(v)) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NegativeDeltas) == 0 { + m.NegativeDeltas = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.NegativeDeltas = append(m.NegativeDeltas, int64(v)) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeDeltas", wireType) + } + case 10: + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.NegativeCounts = append(m.NegativeCounts, v2) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.NegativeCounts) == 0 { + m.NegativeCounts = make([]float64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.NegativeCounts = append(m.NegativeCounts, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeCounts", wireType) + } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveSpans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PositiveSpans = append(m.PositiveSpans, &BucketSpan{}) + if err := m.PositiveSpans[len(m.PositiveSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.PositiveDeltas = append(m.PositiveDeltas, int64(v)) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.PositiveDeltas) == 0 { + m.PositiveDeltas = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.PositiveDeltas = append(m.PositiveDeltas, int64(v)) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveDeltas", wireType) + } + case 13: + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.PositiveCounts = append(m.PositiveCounts, v2) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.PositiveCounts) == 0 { + m.PositiveCounts = make([]float64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.PositiveCounts = append(m.PositiveCounts, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveCounts", wireType) + } + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ResetHint", wireType) + } + m.ResetHint = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ResetHint |= Histogram_ResetHint(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BucketSpan) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BucketSpan: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BucketSpan: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = int32((uint32(v) >> 1) ^ uint32(((v&1)<<31)>>31)) + m.Offset = v + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Length", wireType) + } + m.Length = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Length |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *TimeSeries) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TimeSeries: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TimeSeries: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Labels = append(m.Labels, github_com_thanos_io_thanos_pkg_store_labelpb.ZLabel{}) + if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Samples = append(m.Samples, Sample{}) + if err := m.Samples[len(m.Samples)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Exemplars", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Exemplars = append(m.Exemplars, Exemplar{}) if err := m.Exemplars[len(m.Exemplars)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Histograms", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Histograms = append(m.Histograms, Histogram{}) + if err := m.Histograms[len(m.Histograms)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTypes(dAtA[iNdEx:]) diff --git a/pkg/store/storepb/prompb/types.proto b/pkg/store/storepb/prompb/types.proto index f62ff7552b..0fea825508 100644 --- a/pkg/store/storepb/prompb/types.proto +++ b/pkg/store/storepb/prompb/types.proto @@ -62,6 +62,72 @@ message Exemplar { int64 timestamp = 3; } +// A native histogram, also known as a sparse histogram. +// Original design doc: +// https://docs.google.com/document/d/1cLNv3aufPZb3fNfaJgdaRBZsInZKKIHo9E6HinJVbpM/edit +// The appendix of this design doc also explains the concept of float +// histograms. This Histogram message can represent both, the usual +// integer histogram as well as a float histogram. +message Histogram { + enum ResetHint { + UNKNOWN = 0; // Need to test for a counter reset explicitly. + YES = 1; // This is the 1st histogram after a counter reset. + NO = 2; // There was no counter reset between this and the previous Histogram. + GAUGE = 3; // This is a gauge histogram where counter resets don't happen. + } + + oneof count { // Count of observations in the histogram. + uint64 count_int = 1; + double count_float = 2; + } + double sum = 3; // Sum of observations in the histogram. + // The schema defines the bucket schema. Currently, valid numbers + // are -4 <= n <= 8. They are all for base-2 bucket schemas, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n logarithmic buckets. Or in other words, each + // bucket boundary is the previous boundary times 2^(2^-n). In the + // future, more bucket schemas may be added using numbers < -4 or > + // 8. + sint32 schema = 4; + double zero_threshold = 5; // Breadth of the zero bucket. + oneof zero_count { // Count in zero bucket. + uint64 zero_count_int = 6; + double zero_count_float = 7; + } + + // Negative Buckets. + repeated BucketSpan negative_spans = 8; + // Use either "negative_deltas" or "negative_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + repeated sint64 negative_deltas = 9; // Count delta of each bucket compared to previous one (or to zero for 1st bucket). + repeated double negative_counts = 10; // Absolute count of each bucket. + + // Positive Buckets. + repeated BucketSpan positive_spans = 11; + // Use either "positive_deltas" or "positive_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + repeated sint64 positive_deltas = 12; // Count delta of each bucket compared to previous one (or to zero for 1st bucket). + repeated double positive_counts = 13; // Absolute count of each bucket. + + ResetHint reset_hint = 14; + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + int64 timestamp = 15; +} + +// A BucketSpan defines a number of consecutive buckets with their +// offset. Logically, it would be more straightforward to include the +// bucket counts in the Span. However, the protobuf representation is +// more compact in the way the data is structured here (with all the +// buckets in a single array separate from the Spans). +message BucketSpan { + sint32 offset = 1; // Gap to previous span, or starting point for 1st span (which can be negative). + uint32 length = 2; // Length of consecutive buckets. +} + + // TimeSeries represents samples and labels for a single time series. message TimeSeries { // Labels have to be sorted by label names and without duplicated label names. @@ -69,6 +135,7 @@ message TimeSeries { repeated thanos.Label labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; repeated Sample samples = 2 [(gogoproto.nullable) = false]; repeated Exemplar exemplars = 3 [(gogoproto.nullable) = false]; + repeated Histogram histograms = 4 [(gogoproto.nullable) = false]; } // Matcher specifies a rule, which can match or set of labels or not. @@ -104,6 +171,7 @@ message Chunk { enum Encoding { UNKNOWN = 0; XOR = 1; + HISTOGRAM = 2; } Encoding type = 3; bytes data = 4; diff --git a/pkg/store/storepb/types.pb.go b/pkg/store/storepb/types.pb.go index 271a785bf5..a87a135ba0 100644 --- a/pkg/store/storepb/types.pb.go +++ b/pkg/store/storepb/types.pb.go @@ -63,15 +63,18 @@ func (PartialResponseStrategy) EnumDescriptor() ([]byte, []int) { type Chunk_Encoding int32 const ( - Chunk_XOR Chunk_Encoding = 0 + Chunk_XOR Chunk_Encoding = 0 + Chunk_HISTOGRAM Chunk_Encoding = 1 ) var Chunk_Encoding_name = map[int32]string{ 0: "XOR", + 1: "HISTOGRAM", } var Chunk_Encoding_value = map[string]int32{ - "XOR": 0, + "XOR": 0, + "HISTOGRAM": 1, } func (x Chunk_Encoding) String() string { @@ -287,41 +290,42 @@ func init() { func init() { proto.RegisterFile("store/storepb/types.proto", fileDescriptor_121fba57de02d8e0) } var fileDescriptor_121fba57de02d8e0 = []byte{ - // 536 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0x4f, 0x6f, 0xd3, 0x4e, - 0x10, 0xf5, 0xda, 0x8e, 0x93, 0xcc, 0xaf, 0x3f, 0x64, 0x96, 0x0a, 0xdc, 0x1e, 0x9c, 0xc8, 0x08, - 0x11, 0x55, 0xaa, 0x2d, 0x15, 0x8e, 0x5c, 0x12, 0x94, 0x1b, 0xb4, 0x74, 0x1b, 0x09, 0xd4, 0x0b, - 0xda, 0xb8, 0x2b, 0xdb, 0x6a, 0xfc, 0x47, 0xf6, 0xba, 0x24, 0xdf, 0x02, 0xc4, 0x9d, 0xcf, 0x93, - 0x63, 0x8f, 0x88, 0x43, 0x04, 0xc9, 0x17, 0x41, 0x1e, 0x3b, 0x40, 0xa4, 0x5c, 0xac, 0xf1, 0x7b, - 0x6f, 0x66, 0x76, 0xde, 0xce, 0xc2, 0x51, 0x21, 0xd3, 0x5c, 0x78, 0xf8, 0xcd, 0xa6, 0x9e, 0x5c, - 0x64, 0xa2, 0x70, 0xb3, 0x3c, 0x95, 0x29, 0x35, 0x64, 0xc8, 0x93, 0xb4, 0x38, 0x3e, 0x0c, 0xd2, - 0x20, 0x45, 0xc8, 0xab, 0xa2, 0x9a, 0x3d, 0x6e, 0x12, 0x67, 0x7c, 0x2a, 0x66, 0xbb, 0x89, 0xce, - 0x1d, 0xb4, 0x5e, 0x87, 0x65, 0x72, 0x4b, 0x4f, 0x40, 0xaf, 0x70, 0x8b, 0xf4, 0xc9, 0xe0, 0xc1, - 0xd9, 0x63, 0xb7, 0x2e, 0xe8, 0x22, 0xe9, 0x8e, 0x13, 0x3f, 0xbd, 0x89, 0x92, 0x80, 0xa1, 0x86, - 0x52, 0xd0, 0x6f, 0xb8, 0xe4, 0x96, 0xda, 0x27, 0x83, 0x03, 0x86, 0x31, 0xb5, 0x40, 0x0f, 0x79, - 0x11, 0x5a, 0x5a, 0x9f, 0x0c, 0xf4, 0x91, 0xbe, 0x5c, 0xf5, 0x08, 0x43, 0xc4, 0x79, 0x04, 0x9d, - 0x6d, 0x3e, 0x6d, 0x83, 0xf6, 0xe1, 0x82, 0x99, 0x8a, 0xf3, 0x8d, 0x80, 0x71, 0x25, 0xf2, 0x48, - 0x14, 0xd4, 0x07, 0x03, 0x4f, 0x56, 0x58, 0xa4, 0xaf, 0x0d, 0xfe, 0x3b, 0xfb, 0x7f, 0xdb, 0xfb, - 0x4d, 0x85, 0x8e, 0x5e, 0x2d, 0x57, 0x3d, 0xe5, 0xc7, 0xaa, 0xf7, 0x32, 0x88, 0x64, 0x58, 0x4e, - 0x5d, 0x3f, 0x8d, 0xbd, 0x5a, 0x70, 0x1a, 0xa5, 0x4d, 0xe4, 0x65, 0xb7, 0x81, 0xb7, 0x33, 0xa4, - 0x7b, 0x8d, 0xd9, 0xac, 0x29, 0x4d, 0x3d, 0x30, 0xfc, 0x6a, 0x94, 0xc2, 0x52, 0xb1, 0xc9, 0xc3, - 0x6d, 0x93, 0x61, 0x10, 0xe4, 0x38, 0x24, 0x9e, 0x59, 0x61, 0x8d, 0xcc, 0xf9, 0xaa, 0x42, 0xf7, - 0x0f, 0x47, 0x8f, 0xa0, 0x13, 0x47, 0xc9, 0x47, 0x19, 0xc5, 0xb5, 0x43, 0x1a, 0x6b, 0xc7, 0x51, - 0x32, 0x89, 0x62, 0x81, 0x14, 0x9f, 0xd7, 0x94, 0xda, 0x50, 0x7c, 0x8e, 0x54, 0x0f, 0xb4, 0x9c, - 0x7f, 0x42, 0x4b, 0xfe, 0x19, 0x0b, 0x2b, 0xb2, 0x8a, 0xa1, 0x4f, 0xa1, 0xe5, 0xa7, 0x65, 0x22, - 0x2d, 0x7d, 0x9f, 0xa4, 0xe6, 0xaa, 0x2a, 0x45, 0x19, 0x5b, 0xad, 0xbd, 0x55, 0x8a, 0x32, 0xae, - 0x04, 0x71, 0x94, 0x58, 0xc6, 0x5e, 0x41, 0x1c, 0x25, 0x28, 0xe0, 0x73, 0xab, 0xbd, 0x5f, 0xc0, - 0xe7, 0xf4, 0x39, 0xb4, 0xb1, 0x97, 0xc8, 0xad, 0xce, 0x3e, 0xd1, 0x96, 0x75, 0xbe, 0x10, 0x38, - 0x40, 0x63, 0xdf, 0x72, 0xe9, 0x87, 0x22, 0xa7, 0xa7, 0x3b, 0x6b, 0x73, 0xb4, 0x73, 0x75, 0x8d, - 0xc6, 0x9d, 0x2c, 0x32, 0xf1, 0x77, 0x73, 0x12, 0xde, 0x18, 0xd5, 0x65, 0x18, 0xd3, 0x43, 0x68, - 0xdd, 0xf1, 0x59, 0x29, 0xd0, 0xa7, 0x2e, 0xab, 0x7f, 0x9c, 0x01, 0xe8, 0x55, 0x1e, 0x35, 0x40, - 0x1d, 0x5f, 0x9a, 0x4a, 0xb5, 0x39, 0xe7, 0xe3, 0x4b, 0x93, 0x54, 0x00, 0x1b, 0x9b, 0x2a, 0x02, - 0x6c, 0x6c, 0x6a, 0x27, 0x2e, 0x3c, 0x79, 0xc7, 0x73, 0x19, 0xf1, 0x19, 0x13, 0x45, 0x96, 0x26, - 0x85, 0xb8, 0x92, 0x39, 0x97, 0x22, 0x58, 0xd0, 0x0e, 0xe8, 0xef, 0x87, 0xec, 0xdc, 0x54, 0x68, - 0x17, 0x5a, 0xc3, 0xd1, 0x05, 0x9b, 0x98, 0x64, 0xf4, 0x6c, 0xf9, 0xcb, 0x56, 0x96, 0x6b, 0x9b, - 0xdc, 0xaf, 0x6d, 0xf2, 0x73, 0x6d, 0x93, 0xcf, 0x1b, 0x5b, 0xb9, 0xdf, 0xd8, 0xca, 0xf7, 0x8d, - 0xad, 0x5c, 0xb7, 0x9b, 0xe7, 0x35, 0x35, 0xf0, 0x81, 0xbc, 0xf8, 0x1d, 0x00, 0x00, 0xff, 0xff, - 0x13, 0xe7, 0xb3, 0x25, 0x76, 0x03, 0x00, 0x00, + // 553 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x93, 0xcf, 0x6e, 0xd3, 0x4e, + 0x10, 0xc7, 0xbd, 0xb6, 0xe3, 0x24, 0xf3, 0x6b, 0x7f, 0x32, 0xab, 0x0a, 0xdc, 0x1e, 0x9c, 0xc8, + 0x08, 0x11, 0x55, 0xaa, 0x2d, 0x15, 0x8e, 0x5c, 0x12, 0x14, 0x01, 0x12, 0x6d, 0xe8, 0x26, 0x12, + 0xa8, 0x17, 0xb4, 0x71, 0x57, 0xb6, 0xd5, 0xf8, 0x8f, 0xbc, 0x6b, 0x48, 0x1e, 0x80, 0x3b, 0x88, + 0x3b, 0xcf, 0x93, 0x63, 0x8f, 0x88, 0x43, 0x04, 0xc9, 0x8b, 0x20, 0xaf, 0x1d, 0x20, 0x52, 0x2e, + 0xd6, 0x78, 0x3e, 0xdf, 0x99, 0xd9, 0x99, 0x9d, 0x85, 0x63, 0x2e, 0xd2, 0x9c, 0x79, 0xf2, 0x9b, + 0x4d, 0x3d, 0xb1, 0xc8, 0x18, 0x77, 0xb3, 0x3c, 0x15, 0x29, 0x36, 0x44, 0x48, 0x93, 0x94, 0x9f, + 0x1c, 0x05, 0x69, 0x90, 0x4a, 0x97, 0x57, 0x5a, 0x15, 0x3d, 0xa9, 0x03, 0x67, 0x74, 0xca, 0x66, + 0xbb, 0x81, 0xce, 0x27, 0x04, 0x8d, 0xe7, 0x61, 0x91, 0xdc, 0xe2, 0x53, 0xd0, 0x4b, 0x60, 0xa1, + 0x2e, 0xea, 0xfd, 0x7f, 0x7e, 0xdf, 0xad, 0x32, 0xba, 0x12, 0xba, 0xc3, 0xc4, 0x4f, 0x6f, 0xa2, + 0x24, 0x20, 0x52, 0x83, 0x31, 0xe8, 0x37, 0x54, 0x50, 0x4b, 0xed, 0xa2, 0xde, 0x01, 0x91, 0x36, + 0xb6, 0x40, 0x0f, 0x29, 0x0f, 0x2d, 0xad, 0x8b, 0x7a, 0xfa, 0x40, 0x5f, 0xae, 0x3a, 0x88, 0x48, + 0x8f, 0xe3, 0x40, 0x6b, 0x1b, 0x8f, 0x9b, 0xa0, 0xbd, 0x1b, 0x11, 0x53, 0xc1, 0x87, 0xd0, 0x7e, + 0xf9, 0x6a, 0x3c, 0x19, 0xbd, 0x20, 0xfd, 0x0b, 0x13, 0x39, 0xdf, 0x10, 0x18, 0x63, 0x96, 0x47, + 0x8c, 0x63, 0x1f, 0x0c, 0x79, 0x52, 0x6e, 0xa1, 0xae, 0xd6, 0xfb, 0xef, 0xfc, 0x70, 0x7b, 0x94, + 0xd7, 0xa5, 0x77, 0xf0, 0x6c, 0xb9, 0xea, 0x28, 0x3f, 0x56, 0x9d, 0xa7, 0x41, 0x24, 0xc2, 0x62, + 0xea, 0xfa, 0x69, 0xec, 0x55, 0x82, 0xb3, 0x28, 0xad, 0x2d, 0x2f, 0xbb, 0x0d, 0xbc, 0x9d, 0xa6, + 0xdd, 0x6b, 0x19, 0x4d, 0xea, 0xd4, 0xd8, 0x03, 0xc3, 0x2f, 0x3b, 0xe3, 0x96, 0x2a, 0x8b, 0xdc, + 0xdb, 0x16, 0xe9, 0x07, 0x41, 0x2e, 0x7b, 0x96, 0x2d, 0x28, 0xa4, 0x96, 0x39, 0x5f, 0x55, 0x68, + 0xff, 0x61, 0xf8, 0x18, 0x5a, 0x71, 0x94, 0xbc, 0x17, 0x51, 0x5c, 0x0d, 0x4c, 0x23, 0xcd, 0x38, + 0x4a, 0x26, 0x51, 0xcc, 0x24, 0xa2, 0xf3, 0x0a, 0xa9, 0x35, 0xa2, 0x73, 0x89, 0x3a, 0xa0, 0xe5, + 0xf4, 0xa3, 0x9c, 0xd0, 0x3f, 0x6d, 0xc9, 0x8c, 0xa4, 0x24, 0xf8, 0x21, 0x34, 0xfc, 0xb4, 0x48, + 0x84, 0xa5, 0xef, 0x93, 0x54, 0xac, 0xcc, 0xc2, 0x8b, 0xd8, 0x6a, 0xec, 0xcd, 0xc2, 0x8b, 0xb8, + 0x14, 0xc4, 0x51, 0x62, 0x19, 0x7b, 0x05, 0x71, 0x94, 0x48, 0x01, 0x9d, 0x5b, 0xcd, 0xfd, 0x02, + 0x3a, 0xc7, 0x8f, 0xa1, 0x29, 0x6b, 0xb1, 0xdc, 0x6a, 0xed, 0x13, 0x6d, 0xa9, 0xf3, 0x05, 0xc1, + 0x81, 0x1c, 0xec, 0x05, 0x15, 0x7e, 0xc8, 0x72, 0x7c, 0xb6, 0xb3, 0x45, 0xc7, 0x3b, 0x57, 0x57, + 0x6b, 0xdc, 0xc9, 0x22, 0x63, 0x7f, 0x17, 0x29, 0xa1, 0xf5, 0xa0, 0xda, 0x44, 0xda, 0xf8, 0x08, + 0x1a, 0x1f, 0xe8, 0xac, 0x60, 0x72, 0x4e, 0x6d, 0x52, 0xfd, 0x38, 0x3d, 0xd0, 0xcb, 0x38, 0x6c, + 0x80, 0x3a, 0xbc, 0x32, 0x95, 0x72, 0x91, 0x2e, 0x87, 0x57, 0x26, 0x2a, 0x1d, 0x64, 0x68, 0xaa, + 0xd2, 0x41, 0x86, 0xa6, 0x76, 0xea, 0xc2, 0x83, 0x37, 0x34, 0x17, 0x11, 0x9d, 0x11, 0xc6, 0xb3, + 0x34, 0xe1, 0x6c, 0x2c, 0x72, 0x2a, 0x58, 0xb0, 0xc0, 0x2d, 0xd0, 0xdf, 0xf6, 0xc9, 0xa5, 0xa9, + 0xe0, 0x36, 0x34, 0xfa, 0x83, 0x11, 0x99, 0x98, 0x68, 0xf0, 0x68, 0xf9, 0xcb, 0x56, 0x96, 0x6b, + 0x1b, 0xdd, 0xad, 0x6d, 0xf4, 0x73, 0x6d, 0xa3, 0xcf, 0x1b, 0x5b, 0xb9, 0xdb, 0xd8, 0xca, 0xf7, + 0x8d, 0xad, 0x5c, 0x37, 0xeb, 0xe7, 0x36, 0x35, 0xe4, 0x83, 0x79, 0xf2, 0x3b, 0x00, 0x00, 0xff, + 0xff, 0x44, 0x9d, 0x95, 0xa2, 0x86, 0x03, 0x00, 0x00, } func (m *Chunk) Marshal() (dAtA []byte, err error) { diff --git a/pkg/store/storepb/types.proto b/pkg/store/storepb/types.proto index 3739e23b8f..67c93fa52e 100644 --- a/pkg/store/storepb/types.proto +++ b/pkg/store/storepb/types.proto @@ -23,6 +23,7 @@ option (gogoproto.goproto_sizecache_all) = false; message Chunk { enum Encoding { XOR = 0; + HISTOGRAM = 1; } Encoding type = 1; bytes data = 2; diff --git a/test/e2e/native_histograms_test.go b/test/e2e/native_histograms_test.go index 7c23bef1d6..bec1f35eff 100644 --- a/test/e2e/native_histograms_test.go +++ b/test/e2e/native_histograms_test.go @@ -14,11 +14,15 @@ import ( "github.com/efficientgo/core/testutil" "github.com/efficientgo/e2e" "github.com/go-kit/log" + "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/storage/remote" + "github.com/stretchr/testify/require" + "github.com/thanos-io/thanos/pkg/promclient" + "github.com/thanos-io/thanos/pkg/receive" "github.com/thanos-io/thanos/pkg/runutil" "github.com/thanos-io/thanos/test/e2e/e2ethanos" ) @@ -28,13 +32,15 @@ func TestQueryNativeHistograms(t *testing.T) { testutil.Ok(t, err) t.Cleanup(e2ethanos.CleanScenario(t, e)) - prom, sidecar := e2ethanos.NewPrometheusWithSidecar(e, "prom", e2ethanos.DefaultPromConfig("prom-alone", 0, "", "", e2ethanos.LocalPrometheusTarget), "", "quay.io/prometheus/prometheus:v2.40.5", "", "native-histograms", "remote-write-receiver") - testutil.Ok(t, e2e.StartAndWaitReady(prom, sidecar)) + prom1, sidecar1 := e2ethanos.NewPrometheusWithSidecar(e, "ha1", e2ethanos.DefaultPromConfig("prom-ha", 0, "", "", e2ethanos.LocalPrometheusTarget), "", "quay.io/prometheus/prometheus:v2.40.5", "", "native-histograms", "remote-write-receiver") + prom2, sidecar2 := e2ethanos.NewPrometheusWithSidecar(e, "ha2", e2ethanos.DefaultPromConfig("prom-ha", 1, "", "", e2ethanos.LocalPrometheusTarget), "", "quay.io/prometheus/prometheus:v2.40.5", "", "native-histograms", "remote-write-receiver") + testutil.Ok(t, e2e.StartAndWaitReady(prom1, sidecar1, prom2, sidecar2)) - querier := e2ethanos.NewQuerierBuilder(e, "querier", sidecar.InternalEndpoint("grpc")).Init() + querier := e2ethanos.NewQuerierBuilder(e, "querier", sidecar1.InternalEndpoint("grpc"), sidecar2.InternalEndpoint("grpc")).Init() testutil.Ok(t, e2e.StartAndWaitReady(querier)) - rawRemoteWriteURL := "http://" + prom.Endpoint("http") + "/api/v1/write" + rawRemoteWriteURL1 := "http://" + prom1.Endpoint("http") + "/api/v1/write" + rawRemoteWriteURL2 := "http://" + prom2.Endpoint("http") + "/api/v1/write" ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) t.Cleanup(cancel) @@ -45,17 +51,27 @@ func TestQueryNativeHistograms(t *testing.T) { return ts } - testutil.Ok(t, storeWriteRequest(ctx, rawRemoteWriteURL, nativeHistogramWriteRequest(ts))) + testutil.Ok(t, storeWriteRequest(ctx, rawRemoteWriteURL1, nativeHistogramWriteRequest(ts))) + testutil.Ok(t, storeWriteRequest(ctx, rawRemoteWriteURL2, nativeHistogramWriteRequest(ts))) // Make sure we can query native histogram directly from Prometheus. - queryAndAssertSeries(t, ctx, prom.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{}, []model.Metric{ + queryAndAssertSeries(t, ctx, prom1.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{}, []model.Metric{ + { + "__name__": "test_histogram", + "foo": "bar", + }, + }) + queryAndAssertSeries(t, ctx, prom2.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{}, []model.Metric{ { "__name__": "test_histogram", "foo": "bar", }, }) - queryAndAssertError(t, ctx, querier.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{}, "invalid chunk encoding") + externalLabels := map[string]string{ + "prometheus": "prom-ha", + } + queryAndAssertRawResult(t, ctx, querier.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{Deduplicate: true}, expectedNativeHistogramResult(ts, externalLabels)) } func TestWriteNativeHistograms(t *testing.T) { @@ -63,13 +79,23 @@ func TestWriteNativeHistograms(t *testing.T) { testutil.Ok(t, err) t.Cleanup(e2ethanos.CleanScenario(t, e)) - receiver := e2ethanos.NewReceiveBuilder(e, "receiver").WithIngestionEnabled().Init() - testutil.Ok(t, e2e.StartAndWaitReady(receiver)) + ingestor0 := e2ethanos.NewReceiveBuilder(e, "ingestor0").WithIngestionEnabled().Init() + ingestor1 := e2ethanos.NewReceiveBuilder(e, "ingestor1").WithIngestionEnabled().Init() + + h := receive.HashringConfig{ + Endpoints: []string{ + ingestor0.InternalEndpoint("grpc"), + ingestor1.InternalEndpoint("grpc"), + }, + } - querier := e2ethanos.NewQuerierBuilder(e, "querier", receiver.InternalEndpoint("grpc")).Init() + router0 := e2ethanos.NewReceiveBuilder(e, "router0").WithRouting(2, h).Init() + testutil.Ok(t, e2e.StartAndWaitReady(ingestor0, ingestor1, router0)) + + querier := e2ethanos.NewQuerierBuilder(e, "querier", ingestor0.InternalEndpoint("grpc"), ingestor1.InternalEndpoint("grpc")).WithReplicaLabels("receive").Init() testutil.Ok(t, e2e.StartAndWaitReady(querier)) - rawRemoteWriteURL := "http://" + receiver.Endpoint("remote-write") + "/api/v1/receive" + rawRemoteWriteURL := "http://" + router0.Endpoint("remote-write") + "/api/v1/receive" ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) t.Cleanup(cancel) @@ -82,12 +108,15 @@ func TestWriteNativeHistograms(t *testing.T) { testutil.Ok(t, storeWriteRequest(ctx, rawRemoteWriteURL, nativeHistogramWriteRequest(ts))) - queryAndAssertSeries(t, ctx, querier.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{}, []model.Metric{}) - queryAndAssertSeries(t, ctx, querier.Endpoint("http"), func() string { return "test_sample" }, getTs, promclient.QueryOptions{}, []model.Metric{ + externalLabels := map[string]string{ + "tenant_id": "default-tenant", + } + queryAndAssertRawResult(t, ctx, querier.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{Deduplicate: true}, expectedNativeHistogramResult(ts, externalLabels)) + + queryAndAssertSeries(t, ctx, querier.Endpoint("http"), func() string { return "test_sample" }, getTs, promclient.QueryOptions{Deduplicate: true}, []model.Metric{ { "__name__": "test_sample", "bar": "foo", - "receive": "receive-receiver", "tenant_id": "default-tenant", }, }) @@ -106,7 +135,7 @@ func nativeHistogramWriteRequest(ts time.Time) *prompb.WriteRequest { Count: 5, ZeroCount: 2, Sum: 18.4, - ZeroThreshold: 1e-100, + ZeroThreshold: 0.1, Schema: 1, PositiveSpans: []histogram.Span{ {Offset: 0, Length: 2}, @@ -129,33 +158,107 @@ func nativeHistogramWriteRequest(ts time.Time) *prompb.WriteRequest { } } -func queryAndAssertError(t testing.TB, ctx context.Context, addr string, q func() string, ts func() time.Time, opts promclient.QueryOptions, str string) { +// Code below this line is a workaround until https://github.com/prometheus/common/pull/417 is available. +func expectedNativeHistogramResult(ts time.Time, externalLabels map[string]string) string { + ls := make([]string, 0, len(externalLabels)) + for k, v := range externalLabels { + ls = append(ls, fmt.Sprintf("\"%v\":\"%v\"", k, v)) + } + + return fmt.Sprintf(` +[ + { + "metric": { + "__name__": "test_histogram", + "foo": "bar", + %v + }, + "histogram": [ + %v, + { + "count": "5", + "sum": "18.4", + "buckets": [ + [ + 3, + "-0.1", + "0.1", + "2" + ], + [ + 0, + "0.7071067811865475", + "1", + "1" + ], + [ + 0, + "1", + "1.414213562373095", + "2" + ], + [ + 0, + "2", + "2.82842712474619", + "1" + ], + [ + 0, + "2.82842712474619", + "4", + "1" + ] + ] + } + ] + } +] +`, strings.Join(ls, ","), toPromTs(ts)) +} + +func toPromTs(ts time.Time) string { + promTs := fmt.Sprintf("%.3f", float64(ts.UnixMilli())/1000) + return strings.TrimLeft(promTs, "0") +} + +func queryAndAssertRawResult(t testing.TB, ctx context.Context, addr string, q func() string, ts func() time.Time, opts promclient.QueryOptions, expectedResult string) { t.Helper() - client := promclient.NewDefaultClient() + var result []byte logger := log.NewLogfmtLogger(os.Stdout) logger = log.With(logger, "ts", log.DefaultTimestampUTC) _ = logger.Log( - "caller", "instantQuery", - "msg", fmt.Sprintf("Waiting for result with error containing %q.", str), + "caller", "queryAndAssertRawResult", + "msg", fmt.Sprintf("Waiting for raw results for query %s", q()), ) + testutil.Ok(t, runutil.RetryWithLog(logger, 5*time.Second, ctx.Done(), func() error { - _, _, err := client.QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) - return errorContainsString(t, err, str) + res, err := simpleInstantQueryRawResult(t, ctx, addr, q, ts, opts) + if err != nil { + return err + } + result = res + return nil })) + + require.JSONEq(t, expectedResult, string(result)) } -func errorContainsString(tb testing.TB, err error, str string) error { - tb.Helper() +func simpleInstantQueryRawResult(t testing.TB, ctx context.Context, addr string, q func() string, ts func() time.Time, opts promclient.QueryOptions) ([]byte, error) { + _, rawRes, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) + if err != nil { + return nil, err + } - if err == nil { - return fmt.Errorf("expected error containing string %q, but error is nil", str) + if len(warnings) > 0 { + return nil, errors.Errorf("unexpected warnings %s", warnings) } - if !strings.Contains(err.Error(), str) { - return fmt.Errorf("expected error containing string %q, but got %q", str, err.Error()) + if len(rawRes) <= 0 { + return rawRes, errors.Errorf("unexpected result size, expected %d; result %d: %s", 0, len(rawRes), rawRes) } - return nil + return rawRes, nil } diff --git a/test/e2e/query_test.go b/test/e2e/query_test.go index 1c599413ac..bb61c1dd03 100644 --- a/test/e2e/query_test.go +++ b/test/e2e/query_test.go @@ -1019,7 +1019,7 @@ func instantQuery(t testing.TB, ctx context.Context, addr string, q func() strin } func simpleInstantQuery(t testing.TB, ctx context.Context, addr string, q func() string, ts func() time.Time, opts promclient.QueryOptions, expectedSeriesLen int) (model.Vector, error) { - res, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) + res, _, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) if err != nil { return nil, err } @@ -1048,7 +1048,7 @@ func queryWaitAndAssert(t *testing.T, ctx context.Context, addr string, q func() "msg", fmt.Sprintf("Waiting for %d results for query %s", len(expected), q()), ) testutil.Ok(t, runutil.RetryWithLog(logger, 10*time.Second, ctx.Done(), func() error { - res, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) + res, _, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) if err != nil { return err } From a67e91369f1e87af33dc342a4aac9ffa99175b83 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Tue, 10 Jan 2023 15:24:42 +0100 Subject: [PATCH 2/8] Renamed lastValue of dedup iter and added TODO Signed-off-by: Sebastian Rabenhorst --- pkg/dedup/iter.go | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/pkg/dedup/iter.go b/pkg/dedup/iter.go index 2462b3c3b6..34b7ef0484 100644 --- a/pkg/dedup/iter.go +++ b/pkg/dedup/iter.go @@ -323,7 +323,7 @@ type adjustableSeriesIterator interface { // adjustAtValue allows to adjust value by implementation if needed knowing the last value. This is used by counter // implementation which can adjust for obsolete counter value. - adjustAtValue(lastValue float64) + adjustAtValue(lastFloatValue float64) } type noopAdjustableSeriesIterator struct { @@ -360,11 +360,11 @@ type counterErrAdjustSeriesIterator struct { errAdjust float64 } -func (it *counterErrAdjustSeriesIterator) adjustAtValue(lastValue float64) { +func (it *counterErrAdjustSeriesIterator) adjustAtValue(lastFloatValue float64) { _, v := it.At() - if lastValue > v { + if lastFloatValue > v { // This replica has obsolete value (did not see the correct "end" of counter value before app restart). Adjust. - it.errAdjust += lastValue - v + it.errAdjust += lastFloatValue - v } } @@ -405,6 +405,7 @@ func (it *dedupSeriesIterator) Next() chunkenc.ValueType { if it.useA != lastUseA && isFloatVal { // We switched replicas. // Ensure values are correct bases on value before At. + // TODO(rabenhorst): Investiagte if we also need to implement adjusting histograms here. it.adjustAtValue(lastFloatVal) } }() @@ -487,12 +488,12 @@ func (it *dedupSeriesIterator) lastFloatVal() (float64, bool) { return 0, false } -func (it *dedupSeriesIterator) adjustAtValue(lastValue float64) { +func (it *dedupSeriesIterator) adjustAtValue(lastFloatValue float64) { if it.aval == chunkenc.ValFloat { - it.a.adjustAtValue(lastValue) + it.a.adjustAtValue(lastFloatValue) } if it.bval == chunkenc.ValFloat { - it.b.adjustAtValue(lastValue) + it.b.adjustAtValue(lastFloatValue) } } From 0f28067762f9c9bb90248b5a90538bbf32b9b307 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Tue, 10 Jan 2023 18:07:10 +0100 Subject: [PATCH 3/8] Fixed typo Signed-off-by: Sebastian Rabenhorst --- pkg/dedup/iter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/dedup/iter.go b/pkg/dedup/iter.go index 34b7ef0484..a84ea54767 100644 --- a/pkg/dedup/iter.go +++ b/pkg/dedup/iter.go @@ -405,7 +405,7 @@ func (it *dedupSeriesIterator) Next() chunkenc.ValueType { if it.useA != lastUseA && isFloatVal { // We switched replicas. // Ensure values are correct bases on value before At. - // TODO(rabenhorst): Investiagte if we also need to implement adjusting histograms here. + // TODO(rabenhorst): Investigate if we also need to implement adjusting histograms here. it.adjustAtValue(lastFloatVal) } }() From edfc9e63dbb8fe928a2197c58420a1f3477d1ff0 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Thu, 12 Jan 2023 14:52:38 +0100 Subject: [PATCH 4/8] Added hidden native histogram flag for receive tsdb Signed-off-by: Sebastian Rabenhorst --- cmd/thanos/receive.go | 8 ++++++-- test/e2e/e2ethanos/services.go | 10 ++++++++++ test/e2e/native_histograms_test.go | 4 ++-- 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/cmd/thanos/receive.go b/cmd/thanos/receive.go index e5dcc611ad..3e89a2e170 100644 --- a/cmd/thanos/receive.go +++ b/cmd/thanos/receive.go @@ -88,8 +88,7 @@ func registerReceive(app *extkingpin.App) { EnableExemplarStorage: true, HeadChunksWriteQueueSize: int(conf.tsdbWriteQueueSize), EnableMemorySnapshotOnShutdown: conf.tsdbMemorySnapshotOnShutdown, - //TODO (rabenhorst): Do we want to make this configurable? - EnableNativeHistograms: true, + EnableNativeHistograms: conf.tsdbEnableNativeHistograms, } // Are we running in IngestorOnly, RouterOnly or RouterIngestor mode? @@ -785,6 +784,7 @@ type receiveConfig struct { tsdbMaxExemplars int64 tsdbWriteQueueSize int64 tsdbMemorySnapshotOnShutdown bool + tsdbEnableNativeHistograms bool walCompression bool noLockFile bool @@ -897,6 +897,10 @@ func (rc *receiveConfig) registerFlag(cmd extkingpin.FlagClause) { "[EXPERIMENTAL] Enables feature to snapshot in-memory chunks on shutdown for faster restarts."). Default("false").Hidden().BoolVar(&rc.tsdbMemorySnapshotOnShutdown) + cmd.Flag("tsdb.enable-native-histograms", + "[EXPERIMENTAL] Enables the ingestion of native histograms."). + Default("false").Hidden().BoolVar(&rc.tsdbEnableNativeHistograms) + cmd.Flag("hash-func", "Specify which hash function to use when calculating the hashes of produced files. If no function has been specified, it does not happen. This permits avoiding downloading some files twice albeit at some performance cost. Possible values are: \"\", \"SHA256\"."). Default("").EnumVar(&rc.hashFunc, "SHA256", "") diff --git a/test/e2e/e2ethanos/services.go b/test/e2e/e2ethanos/services.go index 8eb58bcae7..cc46f356f6 100644 --- a/test/e2e/e2ethanos/services.go +++ b/test/e2e/e2ethanos/services.go @@ -455,6 +455,7 @@ type ReceiveBuilder struct { relabelConfigs []*relabel.Config replication int image string + nativeHistograms bool } func NewReceiveBuilder(e e2e.Environment, name string) *ReceiveBuilder { @@ -505,6 +506,11 @@ func (r *ReceiveBuilder) WithValidationEnabled(limit int, metaMonitoring string, return r } +func (r *ReceiveBuilder) WithNativeHistograms() *ReceiveBuilder { + r.nativeHistograms = true + return r +} + // Init creates a Thanos Receive instance. // If ingestion is enabled it will be configured for ingesting samples. // If routing is configured (i.e. hashring configuration is provided) it routes samples to other receivers. @@ -583,6 +589,10 @@ func (r *ReceiveBuilder) Init() *e2emon.InstrumentedRunnable { args["--receive.relabel-config"] = string(relabelConfigBytes) } + if r.nativeHistograms { + args["--tsdb.enable-native-histograms"] = "" + } + return e2emon.AsInstrumented(r.f.Init(wrapWithDefaults(e2e.StartOptions{ Image: r.image, Command: e2e.NewCommand("receive", e2e.BuildArgs(args)...), diff --git a/test/e2e/native_histograms_test.go b/test/e2e/native_histograms_test.go index bec1f35eff..5c03e3dc28 100644 --- a/test/e2e/native_histograms_test.go +++ b/test/e2e/native_histograms_test.go @@ -79,8 +79,8 @@ func TestWriteNativeHistograms(t *testing.T) { testutil.Ok(t, err) t.Cleanup(e2ethanos.CleanScenario(t, e)) - ingestor0 := e2ethanos.NewReceiveBuilder(e, "ingestor0").WithIngestionEnabled().Init() - ingestor1 := e2ethanos.NewReceiveBuilder(e, "ingestor1").WithIngestionEnabled().Init() + ingestor0 := e2ethanos.NewReceiveBuilder(e, "ingestor0").WithIngestionEnabled().WithNativeHistograms().Init() + ingestor1 := e2ethanos.NewReceiveBuilder(e, "ingestor1").WithIngestionEnabled().WithNativeHistograms().Init() h := receive.HashringConfig{ Endpoints: []string{ From 25e26aa982b4cd5fb00583e3e8ef200823872532 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Mon, 16 Jan 2023 14:25:52 +0100 Subject: [PATCH 5/8] merge Signed-off-by: Sebastian Rabenhorst --- go.mod | 2 +- go.sum | 2 + pkg/promclient/promclient.go | 24 +-- pkg/receive/head_series_limiter.go | 2 +- pkg/store/prometheus.go | 2 +- test/e2e/e2ethanos/services.go | 2 +- test/e2e/native_histograms_test.go | 261 +++++++++++------------------ test/e2e/query_test.go | 4 +- 8 files changed, 117 insertions(+), 182 deletions(-) diff --git a/go.mod b/go.mod index 5ec6f1a381..66ae4e7183 100644 --- a/go.mod +++ b/go.mod @@ -62,7 +62,7 @@ require ( github.com/prometheus/alertmanager v0.25.0 github.com/prometheus/client_golang v1.14.0 github.com/prometheus/client_model v0.3.0 - github.com/prometheus/common v0.39.0 + github.com/prometheus/common v0.39.1-0.20230110141620-846591a16635 github.com/prometheus/exporter-toolkit v0.8.2 // Prometheus maps version 2.x.y to tags v0.x.y. github.com/prometheus/prometheus v0.41.0 diff --git a/go.sum b/go.sum index 51081f762f..344f1ed06e 100644 --- a/go.sum +++ b/go.sum @@ -804,6 +804,8 @@ github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+ github.com/prometheus/common v0.37.0/go.mod h1:phzohg0JFMnBEFGxTDbfu3QyL5GI8gTQJFhYO5B3mfA= github.com/prometheus/common v0.39.0 h1:oOyhkDq05hPZKItWVBkJ6g6AtGxi+fy7F4JvUV8uhsI= github.com/prometheus/common v0.39.0/go.mod h1:6XBZ7lYdLCbkAVhwRsWTZn+IN5AB9F/NXd5w0BbEX0Y= +github.com/prometheus/common v0.39.1-0.20230110141620-846591a16635 h1:hK3y58iUBjRFZ6kFNJTWsES1GnVKsqEYUeiyeRXridQ= +github.com/prometheus/common v0.39.1-0.20230110141620-846591a16635/go.mod h1:6XBZ7lYdLCbkAVhwRsWTZn+IN5AB9F/NXd5w0BbEX0Y= github.com/prometheus/common/sigv4 v0.1.0 h1:qoVebwtwwEhS85Czm2dSROY5fTo2PAPEVdDeppTwGX4= github.com/prometheus/common/sigv4 v0.1.0/go.mod h1:2Jkxxk9yYvCkE5G1sQT7GuEXm57JrvHu9k5YwTjsNtI= github.com/prometheus/exporter-toolkit v0.8.2 h1:sbJAfBXQFkG6sUkbwBun8MNdzW9+wd5YfPYofbmj0YM= diff --git a/pkg/promclient/promclient.go b/pkg/promclient/promclient.go index 13e9f71eb7..748ebf189b 100644 --- a/pkg/promclient/promclient.go +++ b/pkg/promclient/promclient.go @@ -385,15 +385,15 @@ func (p *QueryOptions) AddTo(values url.Values) error { } // QueryInstant performs an instant query using a default HTTP client and returns results in model.Vector type. -func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, t time.Time, opts QueryOptions) (model.Vector, json.RawMessage, []string, error) { +func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, t time.Time, opts QueryOptions) (model.Vector, []string, error) { params, err := url.ParseQuery(base.RawQuery) if err != nil { - return nil, nil, nil, errors.Wrapf(err, "parse raw query %s", base.RawQuery) + return nil, nil, errors.Wrapf(err, "parse raw query %s", base.RawQuery) } params.Add("query", query) params.Add("time", t.Format(time.RFC3339Nano)) if err := opts.AddTo(params); err != nil { - return nil, nil, nil, errors.Wrap(err, "add thanos opts query params") + return nil, nil, errors.Wrap(err, "add thanos opts query params") } u := *base @@ -412,7 +412,7 @@ func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, body, _, err := c.req2xx(ctx, &u, method) if err != nil { - return nil, nil, nil, errors.Wrap(err, "read query instant response") + return nil, nil, errors.Wrap(err, "read query instant response") } // Decode only ResultType and load Result only as RawJson since we don't know @@ -430,7 +430,7 @@ func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, } if err = json.Unmarshal(body, &m); err != nil { - return nil, nil, nil, errors.Wrap(err, "unmarshal query instant response") + return nil, nil, errors.Wrap(err, "unmarshal query instant response") } var vectorResult model.Vector @@ -440,29 +440,29 @@ func (c *Client) QueryInstant(ctx context.Context, base *url.URL, query string, switch m.Data.ResultType { case string(parser.ValueTypeVector): if err = json.Unmarshal(m.Data.Result, &vectorResult); err != nil { - return nil, nil, nil, errors.Wrap(err, "decode result into ValueTypeVector") + return nil, nil, errors.Wrap(err, "decode result into ValueTypeVector") } case string(parser.ValueTypeScalar): vectorResult, err = convertScalarJSONToVector(m.Data.Result) if err != nil { - return nil, nil, nil, errors.Wrap(err, "decode result into ValueTypeScalar") + return nil, nil, errors.Wrap(err, "decode result into ValueTypeScalar") } default: if m.Warnings != nil { - return nil, nil, nil, errors.Errorf("error: %s, type: %s, warning: %s", m.Error, m.ErrorType, strings.Join(m.Warnings, ", ")) + return nil, nil, errors.Errorf("error: %s, type: %s, warning: %s", m.Error, m.ErrorType, strings.Join(m.Warnings, ", ")) } if m.Error != "" { - return nil, nil, nil, errors.Errorf("error: %s, type: %s", m.Error, m.ErrorType) + return nil, nil, errors.Errorf("error: %s, type: %s", m.Error, m.ErrorType) } - return nil, nil, nil, errors.Errorf("received status code: 200, unknown response type: '%q'", m.Data.ResultType) + return nil, nil, errors.Errorf("received status code: 200, unknown response type: '%q'", m.Data.ResultType) } - return vectorResult, m.Data.Result, m.Warnings, nil + return vectorResult, m.Warnings, nil } // PromqlQueryInstant performs instant query and returns results in promql.Vector type that is compatible with promql package. func (c *Client) PromqlQueryInstant(ctx context.Context, base *url.URL, query string, t time.Time, opts QueryOptions) (promql.Vector, []string, error) { - vectorResult, _, warnings, err := c.QueryInstant(ctx, base, query, t, opts) + vectorResult, warnings, err := c.QueryInstant(ctx, base, query, t, opts) if err != nil { return nil, nil, err } diff --git a/pkg/receive/head_series_limiter.go b/pkg/receive/head_series_limiter.go index 7536213ef9..746b92d990 100644 --- a/pkg/receive/head_series_limiter.go +++ b/pkg/receive/head_series_limiter.go @@ -106,7 +106,7 @@ func NewHeadSeriesLimit(w WriteLimitsConfig, registerer prometheus.Registerer, l func (h *headSeriesLimit) QueryMetaMonitoring(ctx context.Context) error { c := promclient.NewWithTracingClient(h.logger, h.metaMonitoringClient, httpconfig.ThanosUserAgent) - vectorRes, _, _, err := c.QueryInstant(ctx, h.metaMonitoringURL, h.metaMonitoringQuery, time.Now(), promclient.QueryOptions{}) + vectorRes, _, err := c.QueryInstant(ctx, h.metaMonitoringURL, h.metaMonitoringQuery, time.Now(), promclient.QueryOptions{}) if err != nil { h.metaMonitoringErr.Inc() return err diff --git a/pkg/store/prometheus.go b/pkg/store/prometheus.go index 7dcb3baa14..b4546c7d05 100644 --- a/pkg/store/prometheus.go +++ b/pkg/store/prometheus.go @@ -255,7 +255,7 @@ func (p *PrometheusStore) queryPrometheus(s storepb.Store_SeriesServer, r *store } matrix = result } else { - vector, _, _, err := p.client.QueryInstant(s.Context(), p.base, r.ToPromQL(), timestamp.Time(r.MaxTime), opts) + vector, _, err := p.client.QueryInstant(s.Context(), p.base, r.ToPromQL(), timestamp.Time(r.MaxTime), opts) if err != nil { return err } diff --git a/test/e2e/e2ethanos/services.go b/test/e2e/e2ethanos/services.go index cc46f356f6..d93cf82211 100644 --- a/test/e2e/e2ethanos/services.go +++ b/test/e2e/e2ethanos/services.go @@ -64,7 +64,7 @@ const ( // DefaultPrometheusImage sets default Prometheus image used in e2e service. func DefaultPrometheusImage() string { - return "quay.io/prometheus/prometheus:v2.38.0" + return "quay.io/prometheus/prometheus:v2.41.0" } // DefaultAlertmanagerImage sets default Alertmanager image used in e2e service. diff --git a/test/e2e/native_histograms_test.go b/test/e2e/native_histograms_test.go index 5c03e3dc28..905f7b9c0e 100644 --- a/test/e2e/native_histograms_test.go +++ b/test/e2e/native_histograms_test.go @@ -5,25 +5,17 @@ package e2e_test import ( "context" - "fmt" - "os" - "strings" "testing" "time" "github.com/efficientgo/core/testutil" "github.com/efficientgo/e2e" - "github.com/go-kit/log" - "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/storage/remote" - "github.com/stretchr/testify/require" - "github.com/thanos-io/thanos/pkg/promclient" "github.com/thanos-io/thanos/pkg/receive" - "github.com/thanos-io/thanos/pkg/runutil" "github.com/thanos-io/thanos/test/e2e/e2ethanos" ) @@ -32,8 +24,8 @@ func TestQueryNativeHistograms(t *testing.T) { testutil.Ok(t, err) t.Cleanup(e2ethanos.CleanScenario(t, e)) - prom1, sidecar1 := e2ethanos.NewPrometheusWithSidecar(e, "ha1", e2ethanos.DefaultPromConfig("prom-ha", 0, "", "", e2ethanos.LocalPrometheusTarget), "", "quay.io/prometheus/prometheus:v2.40.5", "", "native-histograms", "remote-write-receiver") - prom2, sidecar2 := e2ethanos.NewPrometheusWithSidecar(e, "ha2", e2ethanos.DefaultPromConfig("prom-ha", 1, "", "", e2ethanos.LocalPrometheusTarget), "", "quay.io/prometheus/prometheus:v2.40.5", "", "native-histograms", "remote-write-receiver") + prom1, sidecar1 := e2ethanos.NewPrometheusWithSidecar(e, "ha1", e2ethanos.DefaultPromConfig("prom-ha", 0, "", "", e2ethanos.LocalPrometheusTarget), "", e2ethanos.DefaultPrometheusImage(), "", "native-histograms", "remote-write-receiver") + prom2, sidecar2 := e2ethanos.NewPrometheusWithSidecar(e, "ha2", e2ethanos.DefaultPromConfig("prom-ha", 1, "", "", e2ethanos.LocalPrometheusTarget), "", e2ethanos.DefaultPrometheusImage(), "", "native-histograms", "remote-write-receiver") testutil.Ok(t, e2e.StartAndWaitReady(prom1, sidecar1, prom2, sidecar2)) querier := e2ethanos.NewQuerierBuilder(e, "querier", sidecar1.InternalEndpoint("grpc"), sidecar2.InternalEndpoint("grpc")).Init() @@ -45,33 +37,42 @@ func TestQueryNativeHistograms(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) t.Cleanup(cancel) - ts := time.Now() - - getTs := func() time.Time { - return ts - } + testutil.Ok(t, synthesizeHistogram(ctx, rawRemoteWriteURL1)) + testutil.Ok(t, synthesizeHistogram(ctx, rawRemoteWriteURL2)) - testutil.Ok(t, storeWriteRequest(ctx, rawRemoteWriteURL1, nativeHistogramWriteRequest(ts))) - testutil.Ok(t, storeWriteRequest(ctx, rawRemoteWriteURL2, nativeHistogramWriteRequest(ts))) - - // Make sure we can query native histogram directly from Prometheus. - queryAndAssertSeries(t, ctx, prom1.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{}, []model.Metric{ + queryAndAssertSeries(t, ctx, prom1.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{}, []model.Metric{ { - "__name__": "test_histogram", + "__name__": "fake_histogram", "foo": "bar", }, }) - queryAndAssertSeries(t, ctx, prom2.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{}, []model.Metric{ + queryAndAssertSeries(t, ctx, prom2.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{}, []model.Metric{ { - "__name__": "test_histogram", + "__name__": "fake_histogram", "foo": "bar", }, }) - externalLabels := map[string]string{ + queryAndAssert(t, ctx, querier.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{Deduplicate: true}, expectedHistogramModelVector(map[string]string{ "prometheus": "prom-ha", - } - queryAndAssertRawResult(t, ctx, querier.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{Deduplicate: true}, expectedNativeHistogramResult(ts, externalLabels)) + })) + + queryAndAssert(t, ctx, querier.Endpoint("http"), func() string { return "histogram_count(fake_histogram)" }, time.Now, promclient.QueryOptions{Deduplicate: true}, model.Vector{ + &model.Sample{ + Value: 5, + Metric: model.Metric{ + "foo": "bar", + "prometheus": "prom-ha", + }, + }, + }) + + queryAndAssert(t, ctx, querier.Endpoint("http"), func() string { return "group(fake_histogram)" }, time.Now, promclient.QueryOptions{Deduplicate: true}, model.Vector{ + &model.Sample{ + Value: 1, + Metric: model.Metric{}, + }, + }) } func TestWriteNativeHistograms(t *testing.T) { @@ -100,165 +101,97 @@ func TestWriteNativeHistograms(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) t.Cleanup(cancel) - ts := time.Now() - - getTs := func() time.Time { - return ts - } - - testutil.Ok(t, storeWriteRequest(ctx, rawRemoteWriteURL, nativeHistogramWriteRequest(ts))) + err = synthesizeHistogram(ctx, rawRemoteWriteURL) + testutil.Ok(t, err) - externalLabels := map[string]string{ + queryAndAssert(t, ctx, querier.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{Deduplicate: true}, expectedHistogramModelVector(map[string]string{ "tenant_id": "default-tenant", - } - queryAndAssertRawResult(t, ctx, querier.Endpoint("http"), func() string { return "test_histogram" }, getTs, promclient.QueryOptions{Deduplicate: true}, expectedNativeHistogramResult(ts, externalLabels)) - - queryAndAssertSeries(t, ctx, querier.Endpoint("http"), func() string { return "test_sample" }, getTs, promclient.QueryOptions{Deduplicate: true}, []model.Metric{ - { - "__name__": "test_sample", - "bar": "foo", - "tenant_id": "default-tenant", - }, - }) + })) } -func nativeHistogramWriteRequest(ts time.Time) *prompb.WriteRequest { - return &prompb.WriteRequest{ - Timeseries: []prompb.TimeSeries{ - { - Labels: []prompb.Label{ - {Name: "__name__", Value: "test_histogram"}, - {Name: "foo", Value: "bar"}, - }, - Histograms: []prompb.Histogram{ - remote.HistogramToHistogramProto(ts.UnixMilli(), &histogram.Histogram{ - Count: 5, - ZeroCount: 2, - Sum: 18.4, - ZeroThreshold: 0.1, - Schema: 1, - PositiveSpans: []histogram.Span{ - {Offset: 0, Length: 2}, - {Offset: 1, Length: 2}, - }, - PositiveBuckets: []int64{1, 1, -1, 0}, // counts: 1, 2, 1, 1 (total 5) - }), - }, - }, - { - Labels: []prompb.Label{ - {Name: "__name__", Value: "test_sample"}, - {Name: "bar", Value: "foo"}, - }, - Samples: []prompb.Sample{ - {Timestamp: ts.UnixMilli(), Value: 1.2}, - }, - }, +func synthesizeHistogram(ctx context.Context, rawRemoteWriteURL string) error { + timeSeriespb := prompb.TimeSeries{ + Labels: []prompb.Label{ + {Name: "__name__", Value: "fake_histogram"}, + {Name: "foo", Value: "bar"}, + }, + Histograms: []prompb.Histogram{ + remote.HistogramToHistogramProto(time.Now().UnixMilli(), testHistogram()), }, } + + return storeWriteRequest(ctx, rawRemoteWriteURL, &prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{timeSeriespb}, + }) } -// Code below this line is a workaround until https://github.com/prometheus/common/pull/417 is available. -func expectedNativeHistogramResult(ts time.Time, externalLabels map[string]string) string { - ls := make([]string, 0, len(externalLabels)) - for k, v := range externalLabels { - ls = append(ls, fmt.Sprintf("\"%v\":\"%v\"", k, v)) +func expectedHistogramModelVector(externalLabels map[string]string) model.Vector { + metrics := model.Metric{ + "__name__": "fake_histogram", + "foo": "bar", + } + for labelKey, labelValue := range externalLabels { + metrics[model.LabelName(labelKey)] = model.LabelValue(labelValue) } - return fmt.Sprintf(` -[ - { - "metric": { - "__name__": "test_histogram", - "foo": "bar", - %v + return model.Vector{ + &model.Sample{ + Metric: metrics, + Histogram: histogramToSampleHistogram(testHistogram()), }, - "histogram": [ - %v, - { - "count": "5", - "sum": "18.4", - "buckets": [ - [ - 3, - "-0.1", - "0.1", - "2" - ], - [ - 0, - "0.7071067811865475", - "1", - "1" - ], - [ - 0, - "1", - "1.414213562373095", - "2" - ], - [ - 0, - "2", - "2.82842712474619", - "1" - ], - [ - 0, - "2.82842712474619", - "4", - "1" - ] - ] - } - ] } -] -`, strings.Join(ls, ","), toPromTs(ts)) } -func toPromTs(ts time.Time) string { - promTs := fmt.Sprintf("%.3f", float64(ts.UnixMilli())/1000) - return strings.TrimLeft(promTs, "0") -} +func histogramToSampleHistogram(h *histogram.Histogram) *model.SampleHistogram { + var buckets []*model.HistogramBucket -func queryAndAssertRawResult(t testing.TB, ctx context.Context, addr string, q func() string, ts func() time.Time, opts promclient.QueryOptions, expectedResult string) { - t.Helper() - - var result []byte - - logger := log.NewLogfmtLogger(os.Stdout) - logger = log.With(logger, "ts", log.DefaultTimestampUTC) - _ = logger.Log( - "caller", "queryAndAssertRawResult", - "msg", fmt.Sprintf("Waiting for raw results for query %s", q()), - ) - - testutil.Ok(t, runutil.RetryWithLog(logger, 5*time.Second, ctx.Done(), func() error { - res, err := simpleInstantQueryRawResult(t, ctx, addr, q, ts, opts) - if err != nil { - return err - } - result = res - return nil - })) + buckets = append(buckets, bucketToSampleHistogramBucket(h.ZeroBucket())) - require.JSONEq(t, expectedResult, string(result)) -} + it := h.PositiveBucketIterator() + for it.Next() { + buckets = append(buckets, bucketToSampleHistogramBucket(it.At())) + } -func simpleInstantQueryRawResult(t testing.TB, ctx context.Context, addr string, q func() string, ts func() time.Time, opts promclient.QueryOptions) ([]byte, error) { - _, rawRes, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) - if err != nil { - return nil, err + return &model.SampleHistogram{ + Count: model.FloatString(h.Count), + Sum: model.FloatString(h.Sum), + Buckets: buckets, } +} - if len(warnings) > 0 { - return nil, errors.Errorf("unexpected warnings %s", warnings) +func bucketToSampleHistogramBucket(bucket histogram.Bucket[uint64]) *model.HistogramBucket { + return &model.HistogramBucket{ + Lower: model.FloatString(bucket.Lower), + Upper: model.FloatString(bucket.Upper), + Count: model.FloatString(bucket.Count), + Boundaries: boundaries(bucket), } +} - if len(rawRes) <= 0 { - return rawRes, errors.Errorf("unexpected result size, expected %d; result %d: %s", 0, len(rawRes), rawRes) +func boundaries(bucket histogram.Bucket[uint64]) int { + switch { + case bucket.UpperInclusive && !bucket.LowerInclusive: + return 0 + case !bucket.UpperInclusive && bucket.LowerInclusive: + return 1 + case !bucket.UpperInclusive && !bucket.LowerInclusive: + return 2 + default: + return 3 } +} - return rawRes, nil +func testHistogram() *histogram.Histogram { + return &histogram.Histogram{ + Count: 5, + ZeroCount: 2, + ZeroThreshold: 0.001, + Sum: 18.4, + Schema: 1, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + PositiveBuckets: []int64{1, 1, -1, 0}, + } } diff --git a/test/e2e/query_test.go b/test/e2e/query_test.go index bb61c1dd03..1c599413ac 100644 --- a/test/e2e/query_test.go +++ b/test/e2e/query_test.go @@ -1019,7 +1019,7 @@ func instantQuery(t testing.TB, ctx context.Context, addr string, q func() strin } func simpleInstantQuery(t testing.TB, ctx context.Context, addr string, q func() string, ts func() time.Time, opts promclient.QueryOptions, expectedSeriesLen int) (model.Vector, error) { - res, _, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) + res, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) if err != nil { return nil, err } @@ -1048,7 +1048,7 @@ func queryWaitAndAssert(t *testing.T, ctx context.Context, addr string, q func() "msg", fmt.Sprintf("Waiting for %d results for query %s", len(expected), q()), ) testutil.Ok(t, runutil.RetryWithLog(logger, 10*time.Second, ctx.Done(), func() error { - res, _, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) + res, warnings, err := promclient.NewDefaultClient().QueryInstant(ctx, urlParse(t, "http://"+addr), q(), ts(), opts) if err != nil { return err } From 44617d83a3011c1fd1450de77b2b7c14e4d2c2ac Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Mon, 16 Jan 2023 17:06:58 +0100 Subject: [PATCH 6/8] Comments and reverted change to qfe Signed-off-by: Sebastian Rabenhorst --- internal/cortex/querier/series/series_set.go | 4 ++-- .../cortex/querier/series/series_set_test.go | 2 +- pkg/query/querier_test.go | 1 + test/e2e/native_histograms_test.go | 18 ++++++------------ 4 files changed, 10 insertions(+), 15 deletions(-) diff --git a/internal/cortex/querier/series/series_set.go b/internal/cortex/querier/series/series_set.go index ff43361b96..d86d3d1f1e 100644 --- a/internal/cortex/querier/series/series_set.go +++ b/internal/cortex/querier/series/series_set.go @@ -239,7 +239,7 @@ func (d DeletedSeriesIterator) Seek(t int64) chunkenc.ValueType { return valueType } - seekedTs := d.itr.AtT() + seekedTs, _ := d.itr.At() if d.isDeleted(seekedTs) { // point we have seeked into is deleted, Next() should find a new non-deleted sample which is after t and seekedTs return d.Next() @@ -268,7 +268,7 @@ func (d DeletedSeriesIterator) AtT() int64 { func (d DeletedSeriesIterator) Next() chunkenc.ValueType { for valueType := d.itr.Next(); valueType != chunkenc.ValNone; valueType = d.itr.Next() { - ts := d.itr.AtT() + ts, _ := d.itr.At() if d.isDeleted(ts) { continue diff --git a/internal/cortex/querier/series/series_set_test.go b/internal/cortex/querier/series/series_set_test.go index 4210ba832e..844d9799a8 100644 --- a/internal/cortex/querier/series/series_set_test.go +++ b/internal/cortex/querier/series/series_set_test.go @@ -145,7 +145,7 @@ func TestDeletedIterator_WithSeek(t *testing.T) { require.Equal(t, c.valueType, it.Seek(c.seek)) if c.valueType != chunkenc.ValNone { - ts := it.AtT() + ts, _ := it.At() require.Equal(t, c.seekedTs, ts) } } diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 3bb8cdba8b..2789d72cce 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -843,6 +843,7 @@ func (s *mockedSeriesIterator) At() (t int64, v float64) { return sample.t, sample.v } +// TODO(rabenhorst): Needs to be implemented for native histogram support. func (s *mockedSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { panic("not implemented") } diff --git a/test/e2e/native_histograms_test.go b/test/e2e/native_histograms_test.go index 905f7b9c0e..30ede9bfd6 100644 --- a/test/e2e/native_histograms_test.go +++ b/test/e2e/native_histograms_test.go @@ -40,23 +40,16 @@ func TestQueryNativeHistograms(t *testing.T) { testutil.Ok(t, synthesizeHistogram(ctx, rawRemoteWriteURL1)) testutil.Ok(t, synthesizeHistogram(ctx, rawRemoteWriteURL2)) - queryAndAssertSeries(t, ctx, prom1.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{}, []model.Metric{ - { - "__name__": "fake_histogram", - "foo": "bar", - }, - }) - queryAndAssertSeries(t, ctx, prom2.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{}, []model.Metric{ - { - "__name__": "fake_histogram", - "foo": "bar", - }, - }) + // Make sure we can query histogram from both Prometheus instances. + queryAndAssert(t, ctx, prom1.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{}, expectedHistogramModelVector(nil)) + queryAndAssert(t, ctx, prom2.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{}, expectedHistogramModelVector(nil)) + // Query deduplicated histogram from Thanos Querier. queryAndAssert(t, ctx, querier.Endpoint("http"), func() string { return "fake_histogram" }, time.Now, promclient.QueryOptions{Deduplicate: true}, expectedHistogramModelVector(map[string]string{ "prometheus": "prom-ha", })) + // Query histogram using histogram_count function and deduplication from Thanos Querier. queryAndAssert(t, ctx, querier.Endpoint("http"), func() string { return "histogram_count(fake_histogram)" }, time.Now, promclient.QueryOptions{Deduplicate: true}, model.Vector{ &model.Sample{ Value: 5, @@ -67,6 +60,7 @@ func TestQueryNativeHistograms(t *testing.T) { }, }) + // Query histogram using group function to test pushdown. queryAndAssert(t, ctx, querier.Endpoint("http"), func() string { return "group(fake_histogram)" }, time.Now, promclient.QueryOptions{Deduplicate: true}, model.Vector{ &model.Sample{ Value: 1, From 908330e111555765a3ab76d6ef8cb73e649c8dff Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Thu, 19 Jan 2023 13:09:34 +0100 Subject: [PATCH 7/8] Go mod tidy Signed-off-by: Sebastian Rabenhorst --- go.sum | 2 -- 1 file changed, 2 deletions(-) diff --git a/go.sum b/go.sum index 344f1ed06e..2fc2e1d4af 100644 --- a/go.sum +++ b/go.sum @@ -802,8 +802,6 @@ github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9 github.com/prometheus/common v0.29.0/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= github.com/prometheus/common v0.37.0/go.mod h1:phzohg0JFMnBEFGxTDbfu3QyL5GI8gTQJFhYO5B3mfA= -github.com/prometheus/common v0.39.0 h1:oOyhkDq05hPZKItWVBkJ6g6AtGxi+fy7F4JvUV8uhsI= -github.com/prometheus/common v0.39.0/go.mod h1:6XBZ7lYdLCbkAVhwRsWTZn+IN5AB9F/NXd5w0BbEX0Y= github.com/prometheus/common v0.39.1-0.20230110141620-846591a16635 h1:hK3y58iUBjRFZ6kFNJTWsES1GnVKsqEYUeiyeRXridQ= github.com/prometheus/common v0.39.1-0.20230110141620-846591a16635/go.mod h1:6XBZ7lYdLCbkAVhwRsWTZn+IN5AB9F/NXd5w0BbEX0Y= github.com/prometheus/common/sigv4 v0.1.0 h1:qoVebwtwwEhS85Czm2dSROY5fTo2PAPEVdDeppTwGX4= From 6bc4626d48e051b721875f86f13ffe1b25484f66 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Tue, 24 Jan 2023 12:01:26 +0100 Subject: [PATCH 8/8] Dedup iter nit Signed-off-by: Sebastian Rabenhorst --- pkg/query/iter.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/query/iter.go b/pkg/query/iter.go index e01aeb49c2..6213d2c031 100644 --- a/pkg/query/iter.go +++ b/pkg/query/iter.go @@ -307,8 +307,7 @@ func (it *chunkSeriesIterator) AtT() int64 { func (it *chunkSeriesIterator) Next() chunkenc.ValueType { lastT := it.AtT() - var valueType chunkenc.ValueType - if valueType = it.chunks[it.i].Next(); valueType != chunkenc.ValNone { + if valueType := it.chunks[it.i].Next(); valueType != chunkenc.ValNone { it.lastVal = valueType return valueType }