diff --git a/pkg/receive/handler.go b/pkg/receive/handler.go index 7b28a7fd12..1797fb471d 100644 --- a/pkg/receive/handler.go +++ b/pkg/receive/handler.go @@ -53,6 +53,7 @@ import ( "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" + "github.com/thanos-io/thanos/pkg/store/storepb/writev2pb" "github.com/thanos-io/thanos/pkg/tenancy" "github.com/thanos-io/thanos/pkg/tracing" ) @@ -91,6 +92,7 @@ var ( type WriteableStoreAsyncClient interface { storepb.WriteableStoreClient RemoteWriteAsync(context.Context, *storepb.WriteRequest, endpointReplica, []int, chan writeResponse, func(error)) + RemoteWriteAsyncV2(context.Context, *storepb.WriteRequestV2, endpointReplica, []int, chan writeResponse, func(error)) } // Options for the web Handler. @@ -474,6 +476,31 @@ func newWriteResponse(seriesIDs []int, err error, er endpointReplica) writeRespo } } +func parseProtoMsg(contentType string) (WriteProtoFullName, error) { + contentType = strings.TrimSpace(contentType) + + parts := strings.Split(contentType, ";") + if parts[0] != appProtoContentType { + return "", fmt.Errorf("expected %v as the first (media) part, got %v content-type", appProtoContentType, contentType) + } + // Parse potential https://www.rfc-editor.org/rfc/rfc9110#parameter + for _, p := range parts[1:] { + pair := strings.Split(p, "=") + if len(pair) != 2 { + return "", fmt.Errorf("as per https://www.rfc-editor.org/rfc/rfc9110#parameter expected parameters to be key-values, got %v in %v content-type", p, contentType) + } + if pair[0] == "proto" { + ret := WriteProtoFullName(pair[1]) + if err := ret.Validate(); err != nil { + return "", fmt.Errorf("got %v content type; %w", contentType, err) + } + return ret, nil + } + } + // No "proto=" parameter, assuming v1. + return WriteProtoFullNameV1, nil +} + func (h *Handler) receiveHTTP(w http.ResponseWriter, r *http.Request) { var err error span, ctx := tracing.StartSpan(r.Context(), "receive_http") @@ -512,6 +539,42 @@ func (h *Handler) receiveHTTP(w http.ResponseWriter, r *http.Request) { return } + contentType := r.Header.Get("Content-Type") + if contentType == "" { + // Don't break yolo 1.0 clients if not needed. + // We could give http.StatusUnsupportedMediaType, but let's assume 1.0 message by default. + contentType = appProtoContentType + } + + msgType, err := parseProtoMsg(contentType) + if err != nil { + level.Error(tLogger).Log("msg", "Error decoding remote write request", "err", err) + http.Error(w, err.Error(), http.StatusUnsupportedMediaType) + return + } + + enc := r.Header.Get("Content-Encoding") + if enc == "" { + // Don't break yolo 1.0 clients if not needed. This is similar to what we did + // before 2.0: https://github.com/prometheus/prometheus/blob/d78253319daa62c8f28ed47e40bafcad2dd8b586/storage/remote/write_handler.go#L62 + // We could give http.StatusUnsupportedMediaType, but let's assume snappy by default. + } else if enc != string(SnappyBlockCompression) { + err := fmt.Errorf("%v encoding (compression) is not accepted by this server; only %v is acceptable", enc, SnappyBlockCompression) + level.Error(tLogger).Log("msg", "Error decoding remote write request", "err", err) + http.Error(w, err.Error(), http.StatusUnsupportedMediaType) + } + + switch msgType { + case WriteProtoFullNameV1: + h.storeV1(ctx, tLogger, w, r, tenantHTTP) + case WriteProtoFullNameV2: + h.storeV2(ctx, tLogger, w, r, tenantHTTP) + default: + } +} + +func (h *Handler) storeV1(ctx context.Context, tLogger log.Logger, w http.ResponseWriter, r *http.Request, tenantHTTP string) { + var err error requestLimiter := h.Limiter.RequestLimiter() // io.ReadAll dynamically adjust the byte slice for read data, starting from 512B. // Since this is receive hot path, grow upfront saving allocations and CPU time. @@ -621,8 +684,10 @@ func (h *Handler) receiveHTTP(w http.ResponseWriter, r *http.Request) { } type requestStats struct { - timeseries int - totalSamples int + timeseries int + totalSamples int + totalExemplars int + totalHistograms int } type tenantRequestStats map[string]requestStats @@ -865,7 +930,7 @@ func (h *Handler) distributeTimeseriesToReplicas( } for _, rn := range replicas { - endpoint, err := h.hashring.GetN(tenant, &ts, rn) + endpoint, err := h.hashring.GetN(tenant, ts.Labels, rn) if err != nil { return nil, nil, err } @@ -1059,6 +1124,36 @@ func (h *Handler) RemoteWrite(ctx context.Context, r *storepb.WriteRequest) (*st } } +// RemoteWriteV2 implements the gRPC remote write handler for storepb.WriteableStore. +func (h *Handler) RemoteWriteV2(ctx context.Context, r *storepb.WriteRequestV2) (*storepb.WriteResponse, error) { + span, ctx := tracing.StartSpan(ctx, "receive_grpc") + defer span.Finish() + + wreq := writev2pb.Request{ + Timeseries: r.Timeseries, + Symbols: r.Symbols, + } + + _, err := h.handleRequestV2(ctx, h.logger, uint64(r.Replica), &wreq, r.Tenant) + if err != nil { + level.Debug(h.logger).Log("msg", "failed to handle request", "err", err) + } + switch errors.Cause(err) { + case nil: + return &storepb.WriteResponse{}, nil + case errNotReady: + return nil, status.Error(codes.Unavailable, err.Error()) + case errUnavailable: + return nil, status.Error(codes.Unavailable, err.Error()) + case errConflict: + return nil, status.Error(codes.AlreadyExists, err.Error()) + case errBadReplica: + return nil, status.Error(codes.InvalidArgument, err.Error()) + default: + return nil, status.Error(codes.Internal, err.Error()) + } +} + // relabel relabels the time series labels in the remote write request. func (h *Handler) relabel(wreq *prompb.WriteRequest) { if len(h.options.RelabelConfigs) == 0 { @@ -1330,6 +1425,10 @@ func (pw *peerWorker) RemoteWrite(ctx context.Context, in *storepb.WriteRequest, return pw.client.RemoteWrite(ctx, in) } +func (pw *peerWorker) RemoteWriteV2(ctx context.Context, in *storepb.WriteRequestV2, opts ...grpc.CallOption) (*storepb.WriteResponse, error) { + return pw.client.RemoteWriteV2(ctx, in) +} + type peerClient interface { storepb.WriteableStoreClient io.Closer @@ -1414,6 +1513,31 @@ func (p *peerWorker) RemoteWriteAsync(ctx context.Context, req *storepb.WriteReq }) } +func (p *peerWorker) RemoteWriteAsyncV2(ctx context.Context, req *storepb.WriteRequestV2, er endpointReplica, seriesIDs []int, responseWriter chan writeResponse, cb func(error)) { + now := time.Now() + p.wp.Go(func() { + p.forwardDelay.Observe(time.Since(now).Seconds()) + + tracing.DoInSpan(ctx, "receive_forward", func(ctx context.Context) { + _, err := p.client.RemoteWriteV2(ctx, req) + responseWriter <- newWriteResponse( + seriesIDs, + errors.Wrapf(err, "forwarding request to endpoint %v", er.endpoint), + er, + ) + if err != nil { + sp := trace.SpanFromContext(ctx) + sp.SetAttributes(attribute.Bool("error", true)) + sp.SetAttributes(attribute.String("error.msg", err.Error())) + } + cb(err) + }, opentracing.Tags{ + "endpoint": er.endpoint, + "replica": er.replica, + }) + }) +} + type peerGroup struct { logger log.Logger dialOpts []grpc.DialOption diff --git a/pkg/receive/handler_test.go b/pkg/receive/handler_test.go index f3a58e128f..0487f61f00 100644 --- a/pkg/receive/handler_test.go +++ b/pkg/receive/handler_test.go @@ -53,6 +53,7 @@ import ( "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" + "github.com/thanos-io/thanos/pkg/store/storepb/writev2pb" "github.com/thanos-io/thanos/pkg/tenancy" ) @@ -111,6 +112,7 @@ func newFakeAppender(appendErr, commitErr, rollbackErr func() error) *fakeAppend } return &fakeAppender{ samples: make(map[storage.SeriesRef][]prompb.Sample), + exemplars: make(map[storage.SeriesRef][]exemplar.Exemplar), appendErr: appendErr, commitErr: commitErr, rollbackErr: rollbackErr, @@ -908,7 +910,7 @@ func TestReceiveWriteRequestLimits(t *testing.T) { // for a given time series, tenant, and replication factor. func endpointHit(t *testing.T, h Hashring, rf uint64, endpoint, tenant string, timeSeries *prompb.TimeSeries) bool { for i := uint64(0); i < rf; i++ { - e, err := h.GetN(tenant, timeSeries, i) + e, err := h.GetN(tenant, timeSeries.Labels, i) if err != nil { t.Fatalf("got unexpected error querying hashring: %v", err) } @@ -973,6 +975,10 @@ func (f *fakeRemoteWriteGRPCServer) RemoteWrite(ctx context.Context, in *storepb return f.h.RemoteWrite(ctx, in) } +func (f *fakeRemoteWriteGRPCServer) RemoteWriteV2(ctx context.Context, in *storepb.WriteRequestV2, opts ...grpc.CallOption) (*storepb.WriteResponse, error) { + return f.h.RemoteWriteV2(ctx, in) +} + func (f *fakeRemoteWriteGRPCServer) RemoteWriteAsync(ctx context.Context, in *storepb.WriteRequest, er endpointReplica, seriesIDs []int, responses chan writeResponse, cb func(error)) { _, err := f.h.RemoteWrite(ctx, in) responses <- writeResponse{ @@ -983,16 +989,33 @@ func (f *fakeRemoteWriteGRPCServer) RemoteWriteAsync(ctx context.Context, in *st cb(err) } +func (f *fakeRemoteWriteGRPCServer) RemoteWriteAsyncV2(ctx context.Context, in *storepb.WriteRequestV2, er endpointReplica, seriesIDs []int, responses chan writeResponse, cb func(error)) { + _, err := f.h.RemoteWriteV2(ctx, in) + responses <- writeResponse{ + er: er, + err: err, + seriesIDs: seriesIDs, + } + cb(err) +} + func (f *fakeRemoteWriteGRPCServer) Close() error { return nil } func BenchmarkHandlerReceiveHTTP(b *testing.B) { - benchmarkHandlerMultiTSDBReceiveRemoteWrite(testutil.NewTB(b)) + // Switch between v1 and v2 by changing the argument. + benchmarkHandlerMultiTSDBReceiveRemoteWrite(testutil.NewTB(b), "v2") } func TestHandlerReceiveHTTP(t *testing.T) { t.Parallel() - benchmarkHandlerMultiTSDBReceiveRemoteWrite(testutil.NewTB(t)) + benchmarkHandlerMultiTSDBReceiveRemoteWrite(testutil.NewTB(t), "v1") +} + +func TestHandlerReceiveHTTPRemoteWriteV2(t *testing.T) { + t.Parallel() + + benchmarkHandlerMultiTSDBReceiveRemoteWrite(testutil.NewTB(t), "v2") } // tsOverrideTenantStorage is storage that overrides timestamp to make it have consistent interval. @@ -1053,6 +1076,34 @@ func serializeSeriesWithOneSample(t testing.TB, series [][]labelpb.ZLabel) []byt return snappy.Encode(nil, body) } +func serializeSeriesWithOneSampleV2(t testing.TB, series [][]labelpb.ZLabel) []byte { + r := &writev2pb.Request{ + Timeseries: make([]writev2pb.TimeSeries, 0, len(series)), + Symbols: make([]string, 0), + } + + buf := make([]uint32, 0, len(series)*2) + st := writev2pb.NewSymbolTable() + for _, s := range series { + refs := st.SymbolizeLabels(labelpb.ZLabelsToPromLabels(s), buf) + r.Timeseries = append(r.Timeseries, writev2pb.TimeSeries{ + LabelsRefs: refs, + Samples: []writev2pb.Sample{ + { + Value: math.MaxFloat64, + Timestamp: math.MinInt64, + }, + }, + }) + } + + r.Symbols = st.Symbols() + + body, err := proto.Marshal(r) + testutil.Ok(t, err) + return snappy.Encode(nil, body) +} + func makeSeriesWithValues(numSeries int) []prompb.TimeSeries { series := make([]prompb.TimeSeries, numSeries) for i := 0; i < numSeries; i++ { @@ -1074,7 +1125,7 @@ func makeSeriesWithValues(numSeries int) []prompb.TimeSeries { return series } -func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB) { +func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB, mode string) { dir := b.TempDir() handlers, _, closeFunc, err := newTestHandlerHashring([]*fakeAppendable{nil}, 1, AlgorithmHashmod, false) @@ -1109,13 +1160,18 @@ func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB) { testutil.Ok(b, m.Flush()) testutil.Ok(b, m.Open()) + serializerFn := serializeSeriesWithOneSample + if mode == "v2" { + serializerFn = serializeSeriesWithOneSampleV2 + } + for _, tcase := range []struct { name string writeRequest []byte }{ { name: "typical labels under 1KB, 500 of them", - writeRequest: serializeSeriesWithOneSample(b, func() [][]labelpb.ZLabel { + writeRequest: serializerFn(b, func() [][]labelpb.ZLabel { series := make([][]labelpb.ZLabel, 500) for s := 0; s < len(series); s++ { lbls := make([]labelpb.ZLabel, 10) @@ -1130,7 +1186,7 @@ func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB) { }, { name: "typical labels under 1KB, 5000 of them", - writeRequest: serializeSeriesWithOneSample(b, func() [][]labelpb.ZLabel { + writeRequest: serializerFn(b, func() [][]labelpb.ZLabel { series := make([][]labelpb.ZLabel, 5000) for s := 0; s < len(series); s++ { lbls := make([]labelpb.ZLabel, 10) @@ -1145,7 +1201,7 @@ func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB) { }, { name: "typical labels under 1KB, 20000 of them", - writeRequest: serializeSeriesWithOneSample(b, func() [][]labelpb.ZLabel { + writeRequest: serializerFn(b, func() [][]labelpb.ZLabel { series := make([][]labelpb.ZLabel, 20000) for s := 0; s < len(series); s++ { lbls := make([]labelpb.ZLabel, 10) @@ -1160,7 +1216,7 @@ func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB) { }, { name: "extremely large label value 10MB, 10 of them", - writeRequest: serializeSeriesWithOneSample(b, func() [][]labelpb.ZLabel { + writeRequest: serializerFn(b, func() [][]labelpb.ZLabel { series := make([][]labelpb.ZLabel, 10) for s := 0; s < len(series); s++ { lbl := &strings.Builder{} @@ -1198,7 +1254,13 @@ func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB) { b.ResetTimer() for i := 0; i < n; i++ { r := httptest.NewRecorder() - handler.receiveHTTP(r, &http.Request{ContentLength: int64(len(tcase.writeRequest)), Body: io.NopCloser(bytes.NewReader(tcase.writeRequest))}) + req := &http.Request{ContentLength: int64(len(tcase.writeRequest)), Body: io.NopCloser(bytes.NewReader(tcase.writeRequest)), Header: http.Header{}} + if mode == "v2" { + req.Header.Add("Content-Type", contentTypeHeader(WriteProtoFullNameV2)) + req.Header.Add(versionHeader, version2HeaderValue) + req.Header.Add("Content-Encoding", string(SnappyBlockCompression)) + } + handler.receiveHTTP(r, req) testutil.Equals(b, http.StatusOK, r.Code, "got non 200 error: %v", r.Body.String()) } }) @@ -1222,7 +1284,13 @@ func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB) { // First request should be fine, since we don't change timestamp, rest is wrong. r := httptest.NewRecorder() - handler.receiveHTTP(r, &http.Request{ContentLength: int64(len(tcase.writeRequest)), Body: io.NopCloser(bytes.NewReader(tcase.writeRequest))}) + req := &http.Request{ContentLength: int64(len(tcase.writeRequest)), Body: io.NopCloser(bytes.NewReader(tcase.writeRequest)), Header: http.Header{}} + if mode == "v2" { + req.Header.Add("Content-Type", contentTypeHeader(WriteProtoFullNameV2)) + req.Header.Add(versionHeader, version2HeaderValue) + req.Header.Add("Content-Encoding", string(SnappyBlockCompression)) + } + handler.receiveHTTP(r, req) testutil.Equals(b, http.StatusOK, r.Code, "got non 200 error: %v", r.Body.String()) b.Run("conflict errors", func(b testutil.TB) { @@ -1230,7 +1298,13 @@ func benchmarkHandlerMultiTSDBReceiveRemoteWrite(b testutil.TB) { b.ResetTimer() for i := 0; i < n; i++ { r := httptest.NewRecorder() - handler.receiveHTTP(r, &http.Request{ContentLength: int64(len(tcase.writeRequest)), Body: io.NopCloser(bytes.NewReader(tcase.writeRequest))}) + req := &http.Request{ContentLength: int64(len(tcase.writeRequest)), Body: io.NopCloser(bytes.NewReader(tcase.writeRequest)), Header: http.Header{}} + if mode == "v2" { + req.Header.Add("Content-Type", contentTypeHeader(WriteProtoFullNameV2)) + req.Header.Add(versionHeader, version2HeaderValue) + req.Header.Add("Content-Encoding", string(SnappyBlockCompression)) + } + handler.receiveHTTP(r, req) testutil.Equals(b, http.StatusConflict, r.Code, "%v-%s", i, func() string { b, _ := io.ReadAll(r.Body) return string(b) @@ -1679,6 +1753,276 @@ func TestRelabel(t *testing.T) { } } +func TestRelabelV2(t *testing.T) { + t.Parallel() + + for _, tcase := range []struct { + name string + relabel []*relabel.Config + writeRequest writev2pb.Request + expectedWriteRequest writev2pb.Request + }{ + { + name: "empty relabel configs", + writeRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, // References into symbols array + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + expectedWriteRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + }, + { + name: "has relabel configs but no relabelling applied", + relabel: []*relabel.Config{ + { + SourceLabels: model.LabelNames{"zoo"}, + TargetLabel: "bar", + Regex: relabel.MustNewRegexp("bar"), + Action: relabel.Replace, + Replacement: "baz", + }, + }, + writeRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + expectedWriteRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + }, + { + name: "relabel rewrite existing labels", + relabel: []*relabel.Config{ + { + TargetLabel: "foo", + Action: relabel.Replace, + Regex: relabel.MustNewRegexp(""), + Replacement: "test", + }, + { + TargetLabel: "__name__", + Action: relabel.Replace, + Regex: relabel.MustNewRegexp(""), + Replacement: "foo", + }, + }, + writeRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + expectedWriteRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 2, 3}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "foo", "test"}, + }, + }, + { + name: "relabel drops label", + relabel: []*relabel.Config{ + { + Action: relabel.LabelDrop, + Regex: relabel.MustNewRegexp("foo"), + }, + }, + writeRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + expectedWriteRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric"}, + }, + }, + { + name: "relabel drops time series", + relabel: []*relabel.Config{ + { + SourceLabels: model.LabelNames{"foo"}, + Action: relabel.Drop, + Regex: relabel.MustNewRegexp("bar"), + }, + }, + writeRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + expectedWriteRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{}, + Symbols: []string{""}, + }, + }, + { + name: "relabel rewrite existing exemplar series labels", + relabel: []*relabel.Config{ + { + Action: relabel.LabelDrop, + Regex: relabel.MustNewRegexp("foo"), + }, + }, + writeRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + expectedWriteRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric"}, + }, + }, + { + name: "relabel drops exemplars", + relabel: []*relabel.Config{ + { + SourceLabels: model.LabelNames{"foo"}, + Action: relabel.Drop, + Regex: relabel.MustNewRegexp("bar"), + }, + }, + writeRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Timestamp: 0, + Value: 1, + }, + }, + }, + }, + Symbols: []string{"", "__name__", "test_metric", "foo", "bar"}, + }, + expectedWriteRequest: writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{}, + Symbols: []string{""}, + }, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + h := NewHandler(nil, &Options{ + RelabelConfigs: tcase.relabel, + }) + + st, twreq := h.relabelAndSplitTenant(&tcase.writeRequest, "default") + testutil.Equals(t, tcase.expectedWriteRequest.Symbols, st.Symbols()) + for i, ts := range tcase.expectedWriteRequest.Timeseries { + testutil.Equals(t, ts, twreq["default"][i]) + } + }) + } +} + func TestGetStatsLimitParameter(t *testing.T) { t.Parallel() @@ -1775,12 +2119,12 @@ type hashringSeenTenants struct { seenTenants map[string]struct{} } -func (h *hashringSeenTenants) GetN(tenant string, ts *prompb.TimeSeries, n uint64) (Endpoint, error) { +func (h *hashringSeenTenants) GetN(tenant string, lbls []labelpb.ZLabel, n uint64) (Endpoint, error) { if h.seenTenants == nil { h.seenTenants = map[string]struct{}{} } h.seenTenants[tenant] = struct{}{} - return h.Hashring.GetN(tenant, ts, n) + return h.Hashring.GetN(tenant, lbls, n) } func TestDistributeSeries(t *testing.T) { @@ -1871,6 +2215,66 @@ func TestHandlerSplitTenantLabelLocalWrite(t *testing.T) { require.Equal(t, map[string]struct{}{"bar": {}, "foo": {}}, hr.seenTenants) } +func TestHandlerSplitTenantLabelLocalWriteV2(t *testing.T) { + const tenantIDLabelName = "thanos_tenant_id" + + appendable := &fakeAppendable{ + appender: newFakeAppender(nil, nil, nil), + } + + h := NewHandler(nil, &Options{ + Endpoint: "localhost", + SplitTenantLabelName: tenantIDLabelName, + ReceiverMode: RouterIngestor, + ReplicationFactor: 1, + ForwardTimeout: 1 * time.Second, + Writer: NewWriter( + log.NewNopLogger(), + newFakeTenantAppendable(appendable), + &WriterOptions{}, + ), + }) + + // initialize hashring with a single local endpoint matching the handler endpoint to force + // using local write + hashring, err := newSimpleHashring([]Endpoint{ + { + Address: h.options.Endpoint, + }, + }) + require.NoError(t, err) + hr := &hashringSeenTenants{Hashring: hashring} + h.Hashring(hr) + + response, err := h.RemoteWriteV2(context.Background(), &storepb.WriteRequestV2{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Value: 123.45, + Timestamp: time.Now().UnixMilli(), + }, + }, + }, + { + LabelsRefs: []uint32{2, 1, 3, 5}, + Samples: []writev2pb.Sample{ + { + Value: 124.45, + Timestamp: time.Now().UnixMilli(), + }, + }, + }, + }, + Symbols: []string{"", "a", "b", tenantIDLabelName, "bar", "foo"}, + }) + + require.NoError(t, err) + require.NotNil(t, response) + require.Equal(t, map[string]struct{}{"bar": {}, "foo": {}}, hr.seenTenants) +} + func TestHandlerFlippingHashrings(t *testing.T) { t.Parallel() @@ -1948,3 +2352,90 @@ func TestHandlerFlippingHashrings(t *testing.T) { cancel() wg.Wait() } + +func TestHandlerRemoteWriteV2(t *testing.T) { + t.Parallel() + + appendable := &fakeAppendable{ + appender: newFakeAppender(nil, nil, nil), + } + + h := NewHandler(nil, &Options{ + Endpoint: "localhost", + ReceiverMode: RouterIngestor, + ReplicationFactor: 1, + ForwardTimeout: 1 * time.Second, + Writer: NewWriter( + log.NewNopLogger(), + newFakeTenantAppendable(appendable), + &WriterOptions{}, + ), + }) + + hashring, err := newSimpleHashring([]Endpoint{ + { + Address: h.options.Endpoint, + }, + }) + require.NoError(t, err) + hr := &hashringSeenTenants{Hashring: hashring} + h.Hashring(hr) + + for _, tc := range []struct { + name string + request *writev2pb.Request + }{ + { + name: "simple timeseries with samples", + request: &writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{3, 4, 1, 2}, + Samples: []writev2pb.Sample{ + { + Value: 123.45, + Timestamp: time.Now().UnixMilli(), + }, + }, + }, + }, + Symbols: []string{"", "foo", "bar", "__name__", "test_metric"}, + }, + }, + { + name: "timeseries with exemplar", + request: &writev2pb.Request{ + Timeseries: []writev2pb.TimeSeries{ + { + LabelsRefs: []uint32{1, 2, 3, 4}, + Samples: []writev2pb.Sample{ + { + Value: 123.45, + Timestamp: time.Now().UnixMilli(), + }, + }, + Exemplars: []writev2pb.Exemplar{ + { + LabelsRefs: []uint32{2, 3}, + Value: 123.45, + Timestamp: time.Now().UnixMilli(), + }, + }, + }, + }, + Symbols: []string{"", "bar", "baz", "__name__", "test_exemplar"}, + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + response, err := h.RemoteWriteV2(context.Background(), &storepb.WriteRequestV2{ + Tenant: "test", + Timeseries: tc.request.Timeseries, + Symbols: tc.request.Symbols, + }) + + require.NoError(t, err) + require.NotNil(t, response) + }) + } +} diff --git a/pkg/receive/handler_writev2.go b/pkg/receive/handler_writev2.go new file mode 100644 index 0000000000..8a5dc8aeec --- /dev/null +++ b/pkg/receive/handler_writev2.go @@ -0,0 +1,547 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package receive + +import ( + "bytes" + "context" + "io" + "net/http" + "strconv" + "sync" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/gogo/protobuf/proto" + "github.com/klauspost/compress/s2" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/relabel" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "github.com/thanos-io/thanos/pkg/server/http/middleware" + "github.com/thanos-io/thanos/pkg/store/labelpb" + "github.com/thanos-io/thanos/pkg/store/storepb" + "github.com/thanos-io/thanos/pkg/store/storepb/writev2pb" + "github.com/thanos-io/thanos/pkg/tracing" +) + +func (h *Handler) storeV2(ctx context.Context, tLogger log.Logger, w http.ResponseWriter, r *http.Request, tenantHTTP string) { + var err error + requestLimiter := h.Limiter.RequestLimiter() + // io.ReadAll dynamically adjust the byte slice for read data, starting from 512B. + // Since this is receive hot path, grow upfront saving allocations and CPU time. + compressed := bytes.Buffer{} + if r.ContentLength >= 0 { + if !requestLimiter.AllowSizeBytes(tenantHTTP, r.ContentLength) { + http.Error(w, "write request too large", http.StatusRequestEntityTooLarge) + return + } + compressed.Grow(int(r.ContentLength)) + } else { + compressed.Grow(512) + } + _, err = io.Copy(&compressed, r.Body) + if err != nil { + http.Error(w, errors.Wrap(err, "read compressed request body").Error(), http.StatusInternalServerError) + return + } + reqBuf, err := s2.Decode(nil, compressed.Bytes()) + if err != nil { + level.Error(tLogger).Log("msg", "snappy decode error", "err", err) + http.Error(w, errors.Wrap(err, "snappy decode error").Error(), http.StatusBadRequest) + return + } + + if !requestLimiter.AllowSizeBytes(tenantHTTP, int64(len(reqBuf))) { + http.Error(w, "write request too large", http.StatusRequestEntityTooLarge) + return + } + + var wreq writev2pb.Request + if err := proto.Unmarshal(reqBuf, &wreq); err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + rep := uint64(0) + // If the header is empty, we assume the request is not yet replicated. + if replicaRaw := r.Header.Get(h.options.ReplicaHeader); replicaRaw != "" { + if rep, err = strconv.ParseUint(replicaRaw, 10, 64); err != nil { + http.Error(w, "could not parse replica header", http.StatusBadRequest) + return + } + } + + if !requestLimiter.AllowSeries(tenantHTTP, int64(len(wreq.Timeseries))) { + http.Error(w, "too many timeseries", http.StatusRequestEntityTooLarge) + return + } + + totalSamples := 0 + for _, timeseries := range wreq.Timeseries { + totalSamples += len(timeseries.Samples) + } + + if !requestLimiter.AllowSamples(tenantHTTP, int64(totalSamples)) { + http.Error(w, "too many samples", http.StatusRequestEntityTooLarge) + return + } + + responseStatusCode := http.StatusOK + tenantStats, err := h.handleRequestV2(ctx, tLogger, rep, &wreq, tenantHTTP) + if err != nil { + level.Debug(tLogger).Log("msg", "failed to handle request", "err", err.Error()) + switch errors.Cause(err) { + case errNotReady: + responseStatusCode = http.StatusServiceUnavailable + case errUnavailable: + responseStatusCode = http.StatusServiceUnavailable + case errConflict: + responseStatusCode = http.StatusConflict + case errBadReplica: + responseStatusCode = http.StatusBadRequest + default: + level.Error(tLogger).Log("err", err, "msg", "internal server error") + responseStatusCode = http.StatusInternalServerError + } + http.Error(w, err.Error(), responseStatusCode) + } + + for tenant, stats := range tenantStats { + w.Header().Set(writtenSamplesHeader, strconv.Itoa(stats.totalSamples)) + w.Header().Set(writtenHistogramsHeader, strconv.Itoa(stats.totalHistograms)) + w.Header().Set(writtenExemplarsHeader, strconv.Itoa(stats.totalExemplars)) + h.writeTimeseriesTotal.WithLabelValues(strconv.Itoa(responseStatusCode), tenant).Observe(float64(stats.timeseries)) + h.writeSamplesTotal.WithLabelValues(strconv.Itoa(responseStatusCode), tenant).Observe(float64(stats.totalSamples)) + } +} + +func (h *Handler) handleRequestV2(ctx context.Context, tLogger log.Logger, rep uint64, wreq *writev2pb.Request, tenantHTTP string) (tenantRequestStats, error) { + symbolTable, tenantWreqs := h.relabelAndSplitTenant(wreq, tenantHTTP) + if len(wreq.Timeseries) == 0 { + level.Debug(tLogger).Log("msg", "remote write request dropped due to relabeling.") + return tenantRequestStats{}, nil + } + + // This replica value is used to detect cycles in cyclic topologies. + // A non-zero value indicates that the request has already been replicated by a previous receive instance. + // For almost all users, this is only used in fully connected topologies of IngestorRouter instances. + // For acyclic topologies that use RouterOnly and IngestorOnly instances, this causes issues when replicating data. + // See discussion in: https://github.com/thanos-io/thanos/issues/4359. + if h.receiverMode == RouterOnly || h.receiverMode == IngestorOnly { + rep = 0 + } + + // The replica value in the header is one-indexed, thus we need >. + if rep > h.options.ReplicationFactor { + level.Error(tLogger).Log("err", errBadReplica, "msg", "write request rejected", + "request_replica", rep, "replication_factor", h.options.ReplicationFactor) + return tenantRequestStats{}, errBadReplica + } + + r := replica{n: rep, replicated: rep != 0} + + // On the wire, format is 1-indexed and in-code is 0-indexed, so we decrement the value if it was already replicated. + if r.replicated { + r.n-- + } + + // Forward any time series as necessary. All time series + // destined for the local node will be written to the receiver. + // Time series will be replicated as necessary. + return h.fanoutForwardV2(ctx, r, symbolTable, tenantWreqs) +} + +func (h *Handler) gatherWriteStatsV2(rf int, writes ...map[endpointReplica]map[string]trackedV2Series) tenantRequestStats { + stats := make(tenantRequestStats) + for _, write := range writes { + for er := range write { + for tenant, series := range write[er] { + samples := 0 + exemplars := 0 + histograms := 0 + + for _, ts := range series.timeSeries { + samples += len(ts.Samples) + exemplars += len(ts.Exemplars) + histograms += len(ts.Histograms) + } + + if st, ok := stats[tenant]; ok { + st.timeseries += len(series.timeSeries) + st.totalSamples += samples + st.totalExemplars += exemplars + st.totalHistograms += histograms + + stats[tenant] = st + } else { + stats[tenant] = requestStats{ + timeseries: len(series.timeSeries), + totalSamples: samples, + totalExemplars: exemplars, + totalHistograms: histograms, + } + } + } + } + } + + // adjust counters by the replication factor + for tenant, st := range stats { + st.timeseries /= rf + st.totalSamples /= rf + st.totalExemplars /= rf + st.totalHistograms /= rf + stats[tenant] = st + } + + return stats +} + +func (h *Handler) fanoutForwardV2(ctx context.Context, r replica, symbolTable *writev2pb.SymbolsTable, tenantWreqs tenantWreq) (tenantRequestStats, error) { + span, ctx := tracing.StartSpan(ctx, "receive_fanout_forward") + defer span.Finish() + + var replicas []uint64 + if r.replicated { + replicas = []uint64{r.n} + } else { + for rn := uint64(0); rn < h.options.ReplicationFactor; rn++ { + replicas = append(replicas, rn) + } + } + + ctx, cancel := context.WithTimeout(tracing.CopyTraceContext(context.Background(), ctx), h.options.ForwardTimeout) + + var writeErrors writeErrors + + defer func() { + if writeErrors.ErrOrNil() != nil { + // NOTICE: The cancel function is not used on all paths intentionally, + // if there is no error when quorum is reached, + // let forward requests to optimistically run until timeout. + cancel() + } + }() + + logTags := []interface{}{} + for tenant := range tenantWreqs { + logTags = append(logTags, "tenant", tenant) + } + + if id, ok := middleware.RequestIDFromContext(ctx); ok { + logTags = append(logTags, "request-id", id) + } + requestLogger := log.With(h.logger, logTags...) + + localWrites, remoteWrites, err := h.distributeTimeseriesToReplicasV2(symbolTable, tenantWreqs, replicas) + if err != nil { + level.Error(requestLogger).Log("msg", "failed to distribute timeseries to replicas", "err", err) + return tenantRequestStats{}, err + } + + stats := h.gatherWriteStatsV2(len(replicas), localWrites, remoteWrites) + + // Prepare a buffered channel to receive the responses from the local and remote writes. Remote writes will all go + // asynchronously and with this capacity we will never block on writing to the channel. + var maxBufferedResponses int + for er := range localWrites { + maxBufferedResponses += len(localWrites[er]) + } + for er := range remoteWrites { + maxBufferedResponses += len(remoteWrites[er]) + } + + responses := make(chan writeResponse, maxBufferedResponses) + wg := sync.WaitGroup{} + + h.sendWritesV2(ctx, &wg, r.replicated, symbolTable, localWrites, remoteWrites, responses) + + go func() { + wg.Wait() + close(responses) + }() + + // At the end, make sure to exhaust the channel, letting remaining unnecessary requests finish asynchronously. + // This is needed if context is canceled or if we reached success of fail quorum faster. + defer func() { + go func() { + for resp := range responses { + if resp.err != nil { + level.Debug(requestLogger).Log("msg", "request failed, but not needed to achieve quorum", "err", resp.err) + } + } + }() + }() + + quorum := h.writeQuorum() + if r.replicated { + quorum = 1 + } + + totalSeries := 0 + for _, timeseries := range tenantWreqs { + totalSeries += len(timeseries) + } + successes := make([]int, totalSeries) + seriesErrs := newReplicationErrors(quorum, totalSeries) + for { + select { + case <-ctx.Done(): + return stats, ctx.Err() + case resp, hasMore := <-responses: + if !hasMore { + for _, seriesErr := range seriesErrs { + writeErrors.Add(seriesErr) + } + return stats, writeErrors.ErrOrNil() + } + + if resp.err != nil { + // Track errors and successes on a per-series basis. + for _, seriesID := range resp.seriesIDs { + seriesErrs[seriesID].Add(resp.err) + } + + continue + } + // At the end, aggregate all errors if there are any and return them. + for _, seriesID := range resp.seriesIDs { + successes[seriesID]++ + } + if quorumReached(successes, quorum) { + return stats, nil + } + } + } +} + +type trackedV2Series struct { + timeSeries []writev2pb.TimeSeries + seriesIDs []int +} + +// distributeTimeseriesToReplicas distributes the given timeseries from the tenant to different endpoints in a manner +// that achieves the replication factor indicated by replicas. +// The first return value are the series that should be written to the local node. The second return value are the +// series that should be written to remote nodes. +func (h *Handler) distributeTimeseriesToReplicasV2( + symbolTable *writev2pb.SymbolsTable, + tenantWreqs tenantWreq, + replicas []uint64, +) (map[endpointReplica]map[string]trackedV2Series, map[endpointReplica]map[string]trackedV2Series, error) { + h.mtx.RLock() + defer h.mtx.RUnlock() + remoteWrites := make(map[endpointReplica]map[string]trackedV2Series) + localWrites := make(map[endpointReplica]map[string]trackedV2Series) + + b := labels.NewScratchBuilder(0) + + tsIndex := 0 + for tenant, timeseries := range tenantWreqs { + for _, ts := range timeseries { + for _, rn := range replicas { + endpoint, err := h.hashring.GetN(tenant, labelpb.ZLabelsFromPromLabels(writev2pb.DesymbolizeLabels(&b, ts.LabelsRefs, symbolTable.Symbols())), rn) + if err != nil { + return nil, nil, err + } + endpointReplica := endpointReplica{endpoint: endpoint, replica: rn} + var writeDestination = remoteWrites + if endpoint.HasAddress(h.options.Endpoint) { + writeDestination = localWrites + } + writeableSeries, ok := writeDestination[endpointReplica] + if !ok { + writeDestination[endpointReplica] = map[string]trackedV2Series{ + tenant: { + timeSeries: make([]writev2pb.TimeSeries, 0), + seriesIDs: make([]int, 0), + }, + } + } + tenantSeries := writeableSeries[tenant] + tenantSeries.timeSeries = append(tenantSeries.timeSeries, ts) + tenantSeries.seriesIDs = append(tenantSeries.seriesIDs, tsIndex) + writeDestination[endpointReplica][tenant] = tenantSeries + } + tsIndex++ + } + } + + return localWrites, remoteWrites, nil +} + +// sendWrites sends the local and remote writes to execute concurrently, controlling them through the provided sync.WaitGroup. +// The responses from the writes are sent to the responses channel. +func (h *Handler) sendWritesV2( + ctx context.Context, + wg *sync.WaitGroup, + replicated bool, + symbolTable *writev2pb.SymbolsTable, + localWrites map[endpointReplica]map[string]trackedV2Series, + remoteWrites map[endpointReplica]map[string]trackedV2Series, + responses chan writeResponse, +) { + // Do the writes to the local node first. This should be easy and fast. + for writeDestination := range localWrites { + func(writeDestination endpointReplica) { + for tenant, trackedSeries := range localWrites[writeDestination] { + h.sendLocalWriteV2(ctx, symbolTable, writeDestination, tenant, trackedSeries, responses) + } + }(writeDestination) + } + + // Do the writes to remote nodes. Run them all in parallel. + for writeDestination := range remoteWrites { + for tenant, trackedSeries := range remoteWrites[writeDestination] { + wg.Add(1) + h.sendRemoteWriteV2(ctx, tenant, writeDestination, symbolTable, trackedSeries, replicated, responses, wg) + } + } +} + +// sendLocalWrite sends a write request to the local node. +// The responses are sent to the responses channel. +func (h *Handler) sendLocalWriteV2( + ctx context.Context, + symbolTable *writev2pb.SymbolsTable, + writeDestination endpointReplica, + tenantHTTP string, + trackedSeries trackedV2Series, + responses chan<- writeResponse, +) { + span, tracingCtx := tracing.StartSpan(ctx, "receive_local_tsdb_write") + defer span.Finish() + span.SetTag("endpoint", writeDestination.endpoint) + span.SetTag("replica", writeDestination.replica) + + err := h.writer.WriteV2(tracingCtx, tenantHTTP, symbolTable, trackedSeries.timeSeries) + if err != nil { + span.SetTag("error", true) + span.SetTag("error.msg", err.Error()) + responses <- newWriteResponse(trackedSeries.seriesIDs, err, writeDestination) + return + } + + responses <- newWriteResponse(trackedSeries.seriesIDs, nil, writeDestination) + +} + +// sendRemoteWrite sends a write request to the remote node. It takes care of checking whether the endpoint is up or not +// in the peerGroup, correctly marking them as up or down when appropriate. +// The responses are sent to the responses channel. +func (h *Handler) sendRemoteWriteV2( + ctx context.Context, + tenant string, + endpointReplica endpointReplica, + symbolTable *writev2pb.SymbolsTable, + trackedSeries trackedV2Series, + alreadyReplicated bool, + responses chan writeResponse, + wg *sync.WaitGroup, +) { + endpoint := endpointReplica.endpoint + cl, err := h.peers.getConnection(ctx, endpoint) + if err != nil { + if errors.Is(err, errUnavailable) { + err = errors.Wrapf(errUnavailable, "backing off forward request for endpoint %v", endpointReplica) + } + responses <- newWriteResponse(trackedSeries.seriesIDs, err, endpointReplica) + wg.Done() + return + } + + // This is called "real" because it's 1-indexed. + realReplicationIndex := int64(endpointReplica.replica + 1) + // Actually make the request against the endpoint we determined should handle these time series. + cl.RemoteWriteAsyncV2(ctx, &storepb.WriteRequestV2{ + Timeseries: trackedSeries.timeSeries, + Tenant: tenant, + Symbols: symbolTable.Symbols(), + // Increment replica since on-the-wire format is 1-indexed and 0 indicates un-replicated. + Replica: realReplicationIndex, + }, endpointReplica, trackedSeries.seriesIDs, responses, func(err error) { + if err == nil { + h.forwardRequests.WithLabelValues(labelSuccess).Inc() + if !alreadyReplicated { + h.replications.WithLabelValues(labelSuccess).Inc() + } + h.peers.markPeerAvailable(endpoint) + } else { + // Check if peer connection is unavailable, update the peer state to avoid spamming that peer. + if st, ok := status.FromError(err); ok { + if st.Code() == codes.Unavailable { + h.peers.markPeerUnavailable(endpointReplica.endpoint) + } + } + } + wg.Done() + }) +} + +type tenantWreq map[string][]writev2pb.TimeSeries + +// relabel relabels the time series labels in the remote write request. +// It also splits the tenant label if the splitTenantLabelName is set. +// This is the point where we construct a new symbol table for the request, to reflect changes. +// The original symbol table is dropped. +func (h *Handler) relabelAndSplitTenant(wreq *writev2pb.Request, tenantHTTP string) (*writev2pb.SymbolsTable, tenantWreq) { + tenantWreqs := make(tenantWreq) + b := labels.NewScratchBuilder(0) + for _, ts := range wreq.Timeseries { + tenant := tenantHTTP + + if h.splitTenantLabelName != "" { + lbls := writev2pb.DesymbolizeLabels(&b, ts.LabelsRefs, wreq.Symbols) + tenantLabel := lbls.Get(h.splitTenantLabelName) + if tenantLabel != "" { + tenant = tenantLabel + } + } + + tenantWreqs[tenant] = append(tenantWreqs[tenant], ts) + } + + if h.splitTenantLabelName != "" { + h.options.RelabelConfigs = append(h.options.RelabelConfigs, &relabel.Config{ + Action: relabel.LabelDrop, + Regex: relabel.MustNewRegexp(h.splitTenantLabelName), + }) + } + + if len(h.options.RelabelConfigs) == 0 { + st := writev2pb.NewSymbolTableFromSymbols(wreq.Symbols) + wreq.Symbols = wreq.Symbols[:0] + return st, tenantWreqs + } + + st := writev2pb.NewSymbolTable() + relabelledTenantWreqs := make(tenantWreq) + + b = labels.NewScratchBuilder(0) + buf := make([]uint32, 0, len(wreq.Symbols)*2) + + for tenant, timeseries := range tenantWreqs { + relabelledTimeseries := make([]writev2pb.TimeSeries, 0) + for _, ts := range timeseries { + lbls, keep := relabel.Process(writev2pb.DesymbolizeLabels(&b, ts.LabelsRefs, wreq.Symbols), h.options.RelabelConfigs...) + if !keep { + continue + } + + ts.LabelsRefs = st.SymbolizeLabels(lbls, buf) + ts.Metadata.HelpRef, ts.Metadata.UnitRef = st.SymbolizeMetadata(wreq.Symbols[ts.Metadata.HelpRef], wreq.Symbols[ts.Metadata.UnitRef]) + relabelledTimeseries = append(relabelledTimeseries, ts) + } + + if len(relabelledTimeseries) > 0 { + relabelledTenantWreqs[tenant] = relabelledTimeseries + } + } + + wreq.Symbols = wreq.Symbols[:0] + return st, relabelledTenantWreqs +} diff --git a/pkg/receive/hashring.go b/pkg/receive/hashring.go index da919d574b..4eacc41b68 100644 --- a/pkg/receive/hashring.go +++ b/pkg/receive/hashring.go @@ -19,7 +19,6 @@ import ( "github.com/pkg/errors" "github.com/thanos-io/thanos/pkg/store/labelpb" - "github.com/thanos-io/thanos/pkg/store/storepb/prompb" ) // HashringAlgorithm is the algorithm used to distribute series in the ring. @@ -52,9 +51,9 @@ func (i *insufficientNodesError) Error() string { // It returns the node and any error encountered. type Hashring interface { // Get returns the first node that should handle the given tenant and time series. - Get(tenant string, timeSeries *prompb.TimeSeries) (Endpoint, error) + Get(tenant string, timeSeries []labelpb.ZLabel) (Endpoint, error) // GetN returns the nth node that should handle the given tenant and time series. - GetN(tenant string, timeSeries *prompb.TimeSeries, n uint64) (Endpoint, error) + GetN(tenant string, timeSeries []labelpb.ZLabel, n uint64) (Endpoint, error) // Nodes returns a sorted slice of nodes that are in this hashring. Addresses could be duplicated // if, for example, the same address is used for multiple tenants in the multi-hashring. Nodes() []Endpoint @@ -64,8 +63,8 @@ type Hashring interface { type SingleNodeHashring string // Get implements the Hashring interface. -func (s SingleNodeHashring) Get(tenant string, ts *prompb.TimeSeries) (Endpoint, error) { - return s.GetN(tenant, ts, 0) +func (s SingleNodeHashring) Get(tenant string, lbls []labelpb.ZLabel) (Endpoint, error) { + return s.GetN(tenant, lbls, 0) } func (s SingleNodeHashring) Nodes() []Endpoint { @@ -73,7 +72,7 @@ func (s SingleNodeHashring) Nodes() []Endpoint { } // GetN implements the Hashring interface. -func (s SingleNodeHashring) GetN(_ string, _ *prompb.TimeSeries, n uint64) (Endpoint, error) { +func (s SingleNodeHashring) GetN(_ string, _ []labelpb.ZLabel, n uint64) (Endpoint, error) { if n > 0 { return Endpoint{}, &insufficientNodesError{have: 1, want: n + 1} } @@ -104,17 +103,17 @@ func (s simpleHashring) Nodes() []Endpoint { } // Get returns a target to handle the given tenant and time series. -func (s simpleHashring) Get(tenant string, ts *prompb.TimeSeries) (Endpoint, error) { - return s.GetN(tenant, ts, 0) +func (s simpleHashring) Get(tenant string, lbls []labelpb.ZLabel) (Endpoint, error) { + return s.GetN(tenant, lbls, 0) } // GetN returns the nth target to handle the given tenant and time series. -func (s simpleHashring) GetN(tenant string, ts *prompb.TimeSeries, n uint64) (Endpoint, error) { +func (s simpleHashring) GetN(tenant string, lbls []labelpb.ZLabel, n uint64) (Endpoint, error) { if n >= uint64(len(s)) { return Endpoint{}, &insufficientNodesError{have: uint64(len(s)), want: n + 1} } - return s[(labelpb.HashWithPrefix(tenant, ts.Labels)+n)%uint64(len(s))], nil + return s[(labelpb.HashWithPrefix(tenant, lbls)+n)%uint64(len(s))], nil } type section struct { @@ -216,16 +215,16 @@ func calculateSectionReplicas(ringSections sections, replicationFactor uint64, a } } -func (c ketamaHashring) Get(tenant string, ts *prompb.TimeSeries) (Endpoint, error) { - return c.GetN(tenant, ts, 0) +func (c ketamaHashring) Get(tenant string, lbls []labelpb.ZLabel) (Endpoint, error) { + return c.GetN(tenant, lbls, 0) } -func (c ketamaHashring) GetN(tenant string, ts *prompb.TimeSeries, n uint64) (Endpoint, error) { +func (c ketamaHashring) GetN(tenant string, lbls []labelpb.ZLabel, n uint64) (Endpoint, error) { if n >= c.numEndpoints { return Endpoint{}, &insufficientNodesError{have: c.numEndpoints, want: n + 1} } - v := labelpb.HashWithPrefix(tenant, ts.Labels) + v := labelpb.HashWithPrefix(tenant, lbls) var i uint64 i = uint64(sort.Search(len(c.sections), func(i int) bool { @@ -258,17 +257,17 @@ type multiHashring struct { } // Get returns a target to handle the given tenant and time series. -func (m *multiHashring) Get(tenant string, ts *prompb.TimeSeries) (Endpoint, error) { - return m.GetN(tenant, ts, 0) +func (m *multiHashring) Get(tenant string, lbls []labelpb.ZLabel) (Endpoint, error) { + return m.GetN(tenant, lbls, 0) } // GetN returns the nth target to handle the given tenant and time series. -func (m *multiHashring) GetN(tenant string, ts *prompb.TimeSeries, n uint64) (Endpoint, error) { +func (m *multiHashring) GetN(tenant string, lbls []labelpb.ZLabel, n uint64) (Endpoint, error) { m.mu.RLock() h, ok := m.cache[tenant] m.mu.RUnlock() if ok { - return h.GetN(tenant, ts, n) + return h.GetN(tenant, lbls, n) } var found bool @@ -308,7 +307,7 @@ func (m *multiHashring) GetN(tenant string, ts *prompb.TimeSeries, n uint64) (En m.cache[tenant] = m.hashrings[i] m.mu.Unlock() - return m.hashrings[i].GetN(tenant, ts, n) + return m.hashrings[i].GetN(tenant, lbls, n) } } return Endpoint{}, errors.New("no matching hashring to handle tenant") diff --git a/pkg/receive/hashring_test.go b/pkg/receive/hashring_test.go index 0cb2ae4bbe..cbeac0d72d 100644 --- a/pkg/receive/hashring_test.go +++ b/pkg/receive/hashring_test.go @@ -144,7 +144,7 @@ func TestHashringGet(t *testing.T) { hs, err := NewMultiHashring(AlgorithmHashmod, 3, tc.cfg) require.NoError(t, err) - h, err := hs.Get(tc.tenant, ts) + h, err := hs.Get(tc.tenant, ts.Labels) if tc.nodes != nil { if err != nil { t.Errorf("case %q: got unexpected error: %v", tc.name, err) @@ -238,7 +238,7 @@ func TestKetamaHashringGet(t *testing.T) { hashRing, err := newKetamaHashring(test.endpoints, 10, test.n+1) require.NoError(t, err) - result, err := hashRing.GetN("tenant", test.ts, test.n) + result, err := hashRing.GetN("tenant", test.ts.Labels, test.n) require.NoError(t, err) require.Equal(t, test.expectedNode, result.Address) }) @@ -473,7 +473,7 @@ func TestKetamaHashringEvenAZSpread(t *testing.T) { azSpread := make(map[string]int64) for i := 0; i < int(tt.replicas); i++ { - r, err := hashRing.GetN(tenant, ts, uint64(i)) + r, err := hashRing.GetN(tenant, ts.Labels, uint64(i)) testutil.Ok(t, err) for _, n := range tt.nodes { @@ -574,11 +574,10 @@ func TestKetamaHashringEvenNodeSpread(t *testing.T) { nodeSpread := make(map[string]int) for i := 0; i < int(tt.numSeries); i++ { ts := &prompb.TimeSeries{ - Labels: labelpb.ZLabelsFromPromLabels(labels.FromStrings("foo", fmt.Sprintf("%d", i))), - Samples: []prompb.Sample{{Value: 1, Timestamp: 0}}, + Labels: labelpb.ZLabelsFromPromLabels(labels.FromStrings("foo", fmt.Sprintf("%d", i))), } for j := 0; j < int(tt.replicas); j++ { - r, err := hashRing.GetN(tenant, ts, uint64(j)) + r, err := hashRing.GetN(tenant, ts.Labels, uint64(j)) testutil.Ok(t, err) nodeSpread[r.Address]++ @@ -654,7 +653,7 @@ func assignReplicatedSeries(series []prompb.TimeSeries, nodes []Endpoint, replic assignments := make(map[string][]prompb.TimeSeries) for i := uint64(0); i < replicas; i++ { for _, ts := range series { - result, err := hashRing.GetN("tenant", &ts, i) + result, err := hashRing.GetN("tenant", ts.Labels, i) if err != nil { return nil, err } diff --git a/pkg/receive/remote_headers.go b/pkg/receive/remote_headers.go new file mode 100644 index 0000000000..71a3abac1a --- /dev/null +++ b/pkg/receive/remote_headers.go @@ -0,0 +1,82 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package receive + +import ( + "fmt" + "strings" + + "google.golang.org/protobuf/reflect/protoreflect" +) + +// Remote write header constants. +const ( + versionHeader = "X-Prometheus-Remote-Write-Version" + version1HeaderValue = "0.1.0" + version2HeaderValue = "2.0.0" + appProtoContentType = "application/x-protobuf" +) + +type WriteProtoFullName protoreflect.FullName + +const ( + // WriteProtoFullNameV1 represents the `prometheus.WriteRequest` protobuf + // message introduced in the https://prometheus.io/docs/specs/remote_write_spec/. + // To be DEPRECATED. + WriteProtoFullNameV1 WriteProtoFullName = "prometheus.WriteRequest" + // WriteProtoFullNameV2 represents the `io.prometheus.write.v2.Request` protobuf + // message introduced in https://prometheus.io/docs/specs/remote_write_spec_2_0/ + WriteProtoFullNameV2 WriteProtoFullName = "io.prometheus.write.v2.Request" +) + +// Validate returns error if the given reference for the protobuf message is not supported. +func (n WriteProtoFullName) Validate() error { + switch n { + case WriteProtoFullNameV1, WriteProtoFullNameV2: + return nil + default: + return fmt.Errorf("unknown remote write protobuf message %v, supported: %v", n, protoMsgs{WriteProtoFullNameV1, WriteProtoFullNameV2}.String()) + } +} + +type protoMsgs []WriteProtoFullName + +func (m protoMsgs) Strings() []string { + ret := make([]string, 0, len(m)) + for _, typ := range m { + ret = append(ret, string(typ)) + } + return ret +} + +func (m protoMsgs) String() string { + return strings.Join(m.Strings(), ", ") +} + +var contentTypeHeaders = map[WriteProtoFullName]string{ + WriteProtoFullNameV1: appProtoContentType, // Also application/x-protobuf;proto=prometheus.WriteRequest but simplified for compatibility with 1.x spec. + WriteProtoFullNameV2: appProtoContentType + ";proto=io.prometheus.write.v2.Request", +} + +// ContentTypeHeader returns content type header value for the given proto message +// or empty string for unknown proto message. +func contentTypeHeader(m WriteProtoFullName) string { + return contentTypeHeaders[m] +} + +const ( + writtenSamplesHeader = "X-Prometheus-Remote-Write-Samples-Written" + writtenHistogramsHeader = "X-Prometheus-Remote-Write-Histograms-Written" + writtenExemplarsHeader = "X-Prometheus-Remote-Write-Exemplars-Written" +) + +// Compression represents the encoding. Currently remote storage supports only +// one, but we experiment with more, thus leaving the compression scaffolding +// for now. +type Compression string + +const ( + // SnappyBlockCompression represents https://github.com/google/snappy/blob/2c94e11145f0b7b184b831577c93e5a41c4c0346/format_description.txt + SnappyBlockCompression Compression = "snappy" +) diff --git a/pkg/receive/writecapnp/client.go b/pkg/receive/writecapnp/client.go index 3cd9f2d082..22626ea0f0 100644 --- a/pkg/receive/writecapnp/client.go +++ b/pkg/receive/writecapnp/client.go @@ -65,6 +65,11 @@ func (r *RemoteWriteClient) RemoteWrite(ctx context.Context, in *storepb.WriteRe return r.writeWithReconnect(ctx, 2, in) } +// TODO(saswatamcode): Implement this. +func (r *RemoteWriteClient) RemoteWriteV2(ctx context.Context, in *storepb.WriteRequestV2, _ ...grpc.CallOption) (*storepb.WriteResponse, error) { + return &storepb.WriteResponse{}, nil +} + func (r *RemoteWriteClient) writeWithReconnect(ctx context.Context, numReconnects int, in *storepb.WriteRequest) (*storepb.WriteResponse, error) { if err := r.connect(ctx); err != nil { return nil, err diff --git a/pkg/receive/writer.go b/pkg/receive/writer.go index d17fd6453c..b0dad8b337 100644 --- a/pkg/receive/writer.go +++ b/pkg/receive/writer.go @@ -19,6 +19,7 @@ import ( "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb/prompb" + "github.com/thanos-io/thanos/pkg/store/storepb/writev2pb" ) // Appendable returns an Appender. @@ -168,3 +169,92 @@ func (r *Writer) Write(ctx context.Context, tenantID string, wreq []prompb.TimeS } return errs.ErrOrNil() } + +func (r *Writer) WriteV2(ctx context.Context, tenantID string, symbolTable *writev2pb.SymbolsTable, wreq []writev2pb.TimeSeries) error { + tLogger := log.With(r.logger, "tenant", tenantID) + + s, err := r.multiTSDB.TenantAppendable(tenantID) + if err != nil { + return errors.Wrap(err, "get tenant appendable") + } + + app, err := s.Appender(ctx) + if err == tsdb.ErrNotReady { + return err + } + if err != nil { + return errors.Wrap(err, "get appender") + } + getRef := app.(storage.GetRef) + var ( + ref storage.SeriesRef + errorTracker writeErrorTracker + ) + app = &ReceiveAppender{ + tLogger: tLogger, + tooFarInFuture: r.opts.TooFarInFutureTimeWindow, + Appender: app, + } + + b := labels.NewScratchBuilder(0) + for _, t := range wreq { + lset := writev2pb.DesymbolizeLabels(&b, t.LabelsRefs, symbolTable.Symbols()) + lbls := labelpb.ZLabelsFromPromLabels(lset) + // Check if time series labels are valid. If not, skip the time series + // and report the error. + if err := labelpb.ValidateLabels(lbls); err != nil { + lset := &labelpb.ZLabelSet{Labels: lbls} + errorTracker.addLabelsError(err, lset, tLogger) + continue + } + + // Check if the TSDB has cached reference for those labels. + ref, lset = getRef.GetRef(lset, lset.Hash()) + if ref == 0 { + // If not, copy labels, as TSDB will hold those strings long term. Given no + // copy unmarshal we don't want to keep memory for whole protobuf, only for labels. + labelpb.ReAllocZLabelsStrings(&lbls, r.opts.Intern) + lset = labelpb.ZLabelsToPromLabels(lbls) + } + + // Append as many valid samples as possible, but keep track of the errors. + for _, s := range t.Samples { + ref, err = app.Append(ref, lset, s.GetTimestamp(), s.GetValue()) + errorTracker.addSampleError(err, tLogger, lset, s.GetTimestamp(), s.GetValue()) + } + + for _, hp := range t.Histograms { + var ( + h *histogram.Histogram + fh *histogram.FloatHistogram + ) + + if hp.IsFloatHistogram() { + fh = hp.ToFloatHistogram() + } else { + h = hp.ToIntHistogram() + } + + ref, err = app.AppendHistogram(ref, lset, hp.GetTimestamp(), h, fh) + errorTracker.addHistogramError(err, tLogger, lset, hp.GetTimestamp()) + } + + // Current implementation 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 { + for _, ex := range t.Exemplars { + ref, err = app.AppendExemplar(ref, lset, ex.ToExemplar(&b, symbolTable.Symbols())) + errorTracker.addExemplarError(err, tLogger) + } + } + + _, err = app.UpdateMetadata(ref, lset, t.ToMetadata(symbolTable.Symbols())) + errorTracker.addMetadataError(err, tLogger) + } + + errs := errorTracker.collectErrors(tLogger) + if err := app.Commit(); err != nil { + errs.Add(errors.Wrap(err, "commit samples")) + } + return errs.ErrOrNil() +} diff --git a/pkg/receive/writer_errors.go b/pkg/receive/writer_errors.go index cabc626054..0ea9497d98 100644 --- a/pkg/receive/writer_errors.go +++ b/pkg/receive/writer_errors.go @@ -26,6 +26,8 @@ type writeErrorTracker struct { numExemplarsOutOfOrder int numExemplarsDuplicate int numExemplarsLabelLength int + + numMetadataErrors int } func (a *writeErrorTracker) addLabelsError(err error, lset *labelpb.ZLabelSet, logger log.Logger) { @@ -161,3 +163,11 @@ func (a *writeErrorTracker) collectErrors(tLogger log.Logger) writeErrors { } return errs } + +func (a *writeErrorTracker) addMetadataError(err error, tLogger log.Logger) { + if err == nil { + return + } + a.numMetadataErrors++ + level.Debug(tLogger).Log("msg", "Error on ingesting metadata", "err", err) +} diff --git a/pkg/receive/writer_test.go b/pkg/receive/writer_test.go index 2db5e6a341..282ede8b40 100644 --- a/pkg/receive/writer_test.go +++ b/pkg/receive/writer_test.go @@ -27,6 +27,7 @@ import ( "github.com/thanos-io/thanos/pkg/runutil" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb/prompb" + "github.com/thanos-io/thanos/pkg/store/storepb/writev2pb" "github.com/thanos-io/thanos/pkg/tenancy" ) @@ -77,12 +78,13 @@ func TestWriter(t *testing.T) { }, }, }, + // DesymbolizeLabels should sort the labels so this case will never fail. "should error out and skip series with out-of-order labels": { reqs: []*prompb.WriteRequest{ { Timeseries: []prompb.TimeSeries{ { - Labels: append(lbls, labelpb.ZLabel{Name: "a", Value: "1"}, labelpb.ZLabel{Name: "b", Value: "1"}, labelpb.ZLabel{Name: "Z", Value: "1"}, labelpb.ZLabel{Name: "b", Value: "2"}), + Labels: append(lbls, labelpb.ZLabel{Name: "a", Value: "1"}, labelpb.ZLabel{Name: "b", Value: "1"}, labelpb.ZLabel{Name: "Z", Value: "1"}, labelpb.ZLabel{Name: "c", Value: "2"}), Samples: []prompb.Sample{{Value: 1, Timestamp: 10}}, }, }, @@ -103,6 +105,7 @@ func TestWriter(t *testing.T) { }, expectedErr: errors.Wrapf(labelpb.ErrDuplicateLabels, "add 1 series"), }, + // DesymbolizeLabels should sort the labels so this case will never fail. "should error out and skip series with out-of-order labels; accept series with valid labels": { reqs: []*prompb.WriteRequest{ { @@ -381,6 +384,69 @@ func TestWriter(t *testing.T) { assertWrittenData(t, app, testData.expectedIngested) }) + + t.Run("writev2_writer", func(t *testing.T) { + if strings.Contains(testName, "out-of-order labels") { + t.Skip("v2 writer out of order shouldn't happen") + } + logger, m, app := setupMultitsdb(t, testData.maxExemplars) + + w := NewWriter(logger, m, testData.opts) + v2timeseries := make([]writev2pb.TimeSeries, 0, len(testData.reqs)) + + for idx, req := range testData.reqs { + st := writev2pb.NewSymbolTable() + for _, ts := range req.Timeseries { + refs := st.SymbolizeLabels(labelpb.ZLabelsToPromLabels(ts.Labels), nil) + + samples := make([]writev2pb.Sample, 0, len(ts.Samples)) + for _, s := range ts.Samples { + samples = append(samples, writev2pb.Sample{ + Value: s.Value, + Timestamp: s.Timestamp, + }) + } + + exemplars := make([]writev2pb.Exemplar, 0, len(ts.Exemplars)) + for _, e := range ts.Exemplars { + exemplars = append(exemplars, writev2pb.Exemplar{ + LabelsRefs: st.SymbolizeLabels(labelpb.ZLabelsToPromLabels(e.Labels), nil), + Value: e.Value, + Timestamp: e.Timestamp, + }) + } + + histograms := make([]writev2pb.Histogram, 0, len(ts.Histograms)) + for _, h := range ts.Histograms { + if h.IsFloatHistogram() { + histograms = append(histograms, writev2pb.FromFloatHistogram(h.GetTimestamp(), prompb.FloatHistogramProtoToFloatHistogram(h))) + } else { + histograms = append(histograms, writev2pb.FromIntHistogram(h.GetTimestamp(), prompb.HistogramProtoToHistogram(h))) + } + } + + v2timeseries = append(v2timeseries, writev2pb.TimeSeries{ + LabelsRefs: refs, + Samples: samples, + Exemplars: exemplars, + Histograms: histograms, + }) + } + + err := w.WriteV2(context.Background(), tenancy.DefaultTenant, st, v2timeseries) + + // We expect no error on any request except the last one + // which may error (and in that case we assert on it). + if testData.expectedErr == nil || idx < len(testData.reqs)-1 { + testutil.Ok(t, err) + } else { + testutil.NotOk(t, err) + testutil.Equals(t, testData.expectedErr.Error(), err.Error()) + } + } + + assertWrittenData(t, app, testData.expectedIngested) + }) }) } } @@ -504,6 +570,46 @@ func benchmarkWriter(b *testing.B, labelsNum int, seriesNum int, generateHistogr Timeseries: timeSeries, } + st := writev2pb.NewSymbolTable() + v2timeseries := make([]writev2pb.TimeSeries, 0, len(wreq.Timeseries)) + + for _, ts := range wreq.Timeseries { + refs := st.SymbolizeLabels(labelpb.ZLabelsToPromLabels(ts.Labels), nil) + + samples := make([]writev2pb.Sample, 0, len(ts.Samples)) + for _, s := range ts.Samples { + samples = append(samples, writev2pb.Sample{ + Value: s.Value, + Timestamp: s.Timestamp, + }) + } + + exemplars := make([]writev2pb.Exemplar, 0, len(ts.Exemplars)) + for _, e := range ts.Exemplars { + exemplars = append(exemplars, writev2pb.Exemplar{ + LabelsRefs: st.SymbolizeLabels(labelpb.ZLabelsToPromLabels(e.Labels), nil), + Value: e.Value, + Timestamp: e.Timestamp, + }) + } + + histograms := make([]writev2pb.Histogram, 0, len(ts.Histograms)) + for _, h := range ts.Histograms { + if h.IsFloatHistogram() { + histograms = append(histograms, writev2pb.FromFloatHistogram(h.GetTimestamp(), prompb.FloatHistogramProtoToFloatHistogram(h))) + } else { + histograms = append(histograms, writev2pb.FromIntHistogram(h.GetTimestamp(), prompb.HistogramProtoToHistogram(h))) + } + } + + v2timeseries = append(v2timeseries, writev2pb.TimeSeries{ + LabelsRefs: refs, + Samples: samples, + Exemplars: exemplars, + Histograms: histograms, + }) + } + b.Run("without interning", func(b *testing.B) { w := NewWriter(logger, m, &WriterOptions{Intern: false}) @@ -526,6 +632,27 @@ func benchmarkWriter(b *testing.B, labelsNum int, seriesNum int, generateHistogr } }) + b.Run("with interning remote write v2", func(b *testing.B) { + w := NewWriter(logger, m, &WriterOptions{Intern: true}) + + b.ReportAllocs() + b.ResetTimer() + + for i := 0; i < b.N; i++ { + testutil.Ok(b, w.WriteV2(ctx, "foo", st, v2timeseries)) + } + }) + + b.Run("without interning remote write v2", func(b *testing.B) { + w := NewWriter(logger, m, &WriterOptions{Intern: false}) + + b.ReportAllocs() + b.ResetTimer() + + for i := 0; i < b.N; i++ { + testutil.Ok(b, w.WriteV2(ctx, "foo", st, v2timeseries)) + } + }) } // generateLabelsAndSeries generates time series for benchmark with specified number of labels. diff --git a/pkg/store/storepb/rpc.pb.go b/pkg/store/storepb/rpc.pb.go index 9d39f27e14..acb0e56a3a 100644 --- a/pkg/store/storepb/rpc.pb.go +++ b/pkg/store/storepb/rpc.pb.go @@ -14,6 +14,7 @@ import ( proto "github.com/gogo/protobuf/proto" types "github.com/gogo/protobuf/types" prompb "github.com/thanos-io/thanos/pkg/store/storepb/prompb" + writev2pb "github.com/thanos-io/thanos/pkg/store/storepb/writev2pb" grpc "google.golang.org/grpc" codes "google.golang.org/grpc/codes" status "google.golang.org/grpc/status" @@ -142,6 +143,46 @@ func (m *WriteRequest) XXX_DiscardUnknown() { var xxx_messageInfo_WriteRequest proto.InternalMessageInfo +type WriteRequestV2 struct { + Symbols []string `protobuf:"bytes,1,rep,name=symbols,proto3" json:"symbols,omitempty"` + Timeseries []writev2pb.TimeSeries `protobuf:"bytes,2,rep,name=timeseries,proto3" json:"timeseries"` + Tenant string `protobuf:"bytes,3,opt,name=tenant,proto3" json:"tenant,omitempty"` + Replica int64 `protobuf:"varint,4,opt,name=replica,proto3" json:"replica,omitempty"` +} + +func (m *WriteRequestV2) Reset() { *m = WriteRequestV2{} } +func (m *WriteRequestV2) String() string { return proto.CompactTextString(m) } +func (*WriteRequestV2) ProtoMessage() {} +func (*WriteRequestV2) Descriptor() ([]byte, []int) { + return fileDescriptor_a938d55a388af629, []int{2} +} +func (m *WriteRequestV2) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WriteRequestV2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_WriteRequestV2.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 *WriteRequestV2) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteRequestV2.Merge(m, src) +} +func (m *WriteRequestV2) XXX_Size() int { + return m.Size() +} +func (m *WriteRequestV2) XXX_DiscardUnknown() { + xxx_messageInfo_WriteRequestV2.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteRequestV2 proto.InternalMessageInfo + type SeriesRequest struct { MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` @@ -188,7 +229,7 @@ func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } func (m *SeriesRequest) String() string { return proto.CompactTextString(m) } func (*SeriesRequest) ProtoMessage() {} func (*SeriesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{2} + return fileDescriptor_a938d55a388af629, []int{3} } func (m *SeriesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -236,7 +277,7 @@ func (m *QueryHints) Reset() { *m = QueryHints{} } func (m *QueryHints) String() string { return proto.CompactTextString(m) } func (*QueryHints) ProtoMessage() {} func (*QueryHints) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{3} + return fileDescriptor_a938d55a388af629, []int{4} } func (m *QueryHints) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -281,7 +322,7 @@ func (m *ShardInfo) Reset() { *m = ShardInfo{} } func (m *ShardInfo) String() string { return proto.CompactTextString(m) } func (*ShardInfo) ProtoMessage() {} func (*ShardInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{4} + return fileDescriptor_a938d55a388af629, []int{5} } func (m *ShardInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -319,7 +360,7 @@ func (m *Func) Reset() { *m = Func{} } func (m *Func) String() string { return proto.CompactTextString(m) } func (*Func) ProtoMessage() {} func (*Func) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{5} + return fileDescriptor_a938d55a388af629, []int{6} } func (m *Func) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -359,7 +400,7 @@ func (m *Grouping) Reset() { *m = Grouping{} } func (m *Grouping) String() string { return proto.CompactTextString(m) } func (*Grouping) ProtoMessage() {} func (*Grouping) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{6} + return fileDescriptor_a938d55a388af629, []int{7} } func (m *Grouping) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -396,7 +437,7 @@ func (m *Range) Reset() { *m = Range{} } func (m *Range) String() string { return proto.CompactTextString(m) } func (*Range) ProtoMessage() {} func (*Range) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{7} + return fileDescriptor_a938d55a388af629, []int{8} } func (m *Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +478,7 @@ func (m *SeriesResponse) Reset() { *m = SeriesResponse{} } func (m *SeriesResponse) String() string { return proto.CompactTextString(m) } func (*SeriesResponse) ProtoMessage() {} func (*SeriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{8} + return fileDescriptor_a938d55a388af629, []int{9} } func (m *SeriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -544,7 +585,7 @@ func (m *LabelNamesRequest) Reset() { *m = LabelNamesRequest{} } func (m *LabelNamesRequest) String() string { return proto.CompactTextString(m) } func (*LabelNamesRequest) ProtoMessage() {} func (*LabelNamesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{9} + return fileDescriptor_a938d55a388af629, []int{10} } func (m *LabelNamesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -586,7 +627,7 @@ func (m *LabelNamesResponse) Reset() { *m = LabelNamesResponse{} } func (m *LabelNamesResponse) String() string { return proto.CompactTextString(m) } func (*LabelNamesResponse) ProtoMessage() {} func (*LabelNamesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{10} + return fileDescriptor_a938d55a388af629, []int{11} } func (m *LabelNamesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -637,7 +678,7 @@ func (m *LabelValuesRequest) Reset() { *m = LabelValuesRequest{} } func (m *LabelValuesRequest) String() string { return proto.CompactTextString(m) } func (*LabelValuesRequest) ProtoMessage() {} func (*LabelValuesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{11} + return fileDescriptor_a938d55a388af629, []int{12} } func (m *LabelValuesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -679,7 +720,7 @@ func (m *LabelValuesResponse) Reset() { *m = LabelValuesResponse{} } func (m *LabelValuesResponse) String() string { return proto.CompactTextString(m) } func (*LabelValuesResponse) ProtoMessage() {} func (*LabelValuesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_a938d55a388af629, []int{12} + return fileDescriptor_a938d55a388af629, []int{13} } func (m *LabelValuesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -712,6 +753,7 @@ func init() { proto.RegisterEnum("thanos.Aggr", Aggr_name, Aggr_value) proto.RegisterType((*WriteResponse)(nil), "thanos.WriteResponse") proto.RegisterType((*WriteRequest)(nil), "thanos.WriteRequest") + proto.RegisterType((*WriteRequestV2)(nil), "thanos.WriteRequestV2") proto.RegisterType((*SeriesRequest)(nil), "thanos.SeriesRequest") proto.RegisterType((*QueryHints)(nil), "thanos.QueryHints") proto.RegisterType((*ShardInfo)(nil), "thanos.ShardInfo") @@ -728,79 +770,83 @@ func init() { func init() { proto.RegisterFile("store/storepb/rpc.proto", fileDescriptor_a938d55a388af629) } var fileDescriptor_a938d55a388af629 = []byte{ - // 1149 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x56, 0x4b, 0x6f, 0x23, 0x45, - 0x10, 0xf6, 0x78, 0x3c, 0x7e, 0x94, 0x13, 0xaf, 0xb7, 0xd7, 0xc9, 0x4e, 0xbc, 0x92, 0x63, 0x8c, - 0x90, 0xac, 0x55, 0xe4, 0xac, 0xbc, 0x08, 0x09, 0xc4, 0x25, 0x09, 0x2c, 0x59, 0x89, 0x04, 0xe8, - 0xec, 0x12, 0x04, 0x87, 0x51, 0xdb, 0xee, 0x8c, 0x47, 0x3b, 0xaf, 0x4c, 0xf7, 0x90, 0xf8, 0x0c, - 0x67, 0xc4, 0x9d, 0xdb, 0xfe, 0x9a, 0xdc, 0xd8, 0x23, 0x27, 0x04, 0xc9, 0x1f, 0x41, 0xfd, 0x18, - 0x3f, 0x82, 0xf7, 0xa5, 0xe4, 0x62, 0x75, 0x7d, 0x5f, 0x75, 0x4d, 0x75, 0xf5, 0x57, 0xe5, 0x86, - 0xfb, 0x8c, 0x47, 0x09, 0xdd, 0x96, 0xbf, 0xf1, 0x60, 0x3b, 0x89, 0x87, 0xbd, 0x38, 0x89, 0x78, - 0x84, 0x8a, 0x7c, 0x4c, 0xc2, 0x88, 0x35, 0x37, 0x16, 0x1d, 0xf8, 0x24, 0xa6, 0x4c, 0xb9, 0x34, - 0x1b, 0x6e, 0xe4, 0x46, 0x72, 0xb9, 0x2d, 0x56, 0x1a, 0x6d, 0x2f, 0x6e, 0x88, 0x93, 0x28, 0xb8, - 0xb6, 0x6f, 0xc3, 0x8d, 0x22, 0xd7, 0xa7, 0xdb, 0xd2, 0x1a, 0xa4, 0x27, 0xdb, 0x24, 0x9c, 0x28, - 0xaa, 0x73, 0x07, 0x56, 0x8f, 0x13, 0x8f, 0x53, 0x4c, 0x59, 0x1c, 0x85, 0x8c, 0x76, 0x7e, 0x31, - 0x60, 0x45, 0x23, 0xa7, 0x29, 0x65, 0x1c, 0xed, 0x00, 0x70, 0x2f, 0xa0, 0x8c, 0x26, 0x1e, 0x65, - 0xb6, 0xd1, 0x36, 0xbb, 0xd5, 0xfe, 0x03, 0xb1, 0x3b, 0xa0, 0x7c, 0x4c, 0x53, 0xe6, 0x0c, 0xa3, - 0x78, 0xd2, 0x7b, 0xe6, 0x05, 0xf4, 0x48, 0xba, 0xec, 0x16, 0x2e, 0xfe, 0xde, 0xcc, 0xe1, 0xb9, - 0x4d, 0x68, 0x1d, 0x8a, 0x9c, 0x86, 0x24, 0xe4, 0x76, 0xbe, 0x6d, 0x74, 0x2b, 0x58, 0x5b, 0xc8, - 0x86, 0x52, 0x42, 0x63, 0xdf, 0x1b, 0x12, 0xdb, 0x6c, 0x1b, 0x5d, 0x13, 0x67, 0x66, 0xe7, 0xa5, - 0x05, 0xab, 0x2a, 0x5c, 0x96, 0xc6, 0x06, 0x94, 0x03, 0x2f, 0x74, 0x44, 0x54, 0xdb, 0x50, 0xce, - 0x81, 0x17, 0x8a, 0xcf, 0x4a, 0x8a, 0x9c, 0x2b, 0x2a, 0xaf, 0x29, 0x72, 0x2e, 0xa9, 0x4f, 0x04, - 0xc5, 0x87, 0x63, 0x9a, 0x30, 0xdb, 0x94, 0xa9, 0x37, 0x7a, 0xaa, 0xce, 0xbd, 0xaf, 0xc9, 0x80, - 0xfa, 0x07, 0x8a, 0xd4, 0x39, 0x4f, 0x7d, 0x51, 0x1f, 0xd6, 0x44, 0xc8, 0x84, 0xb2, 0xc8, 0x4f, - 0xb9, 0x17, 0x85, 0xce, 0x99, 0x17, 0x8e, 0xa2, 0x33, 0xbb, 0x20, 0xe3, 0xdf, 0x0b, 0xc8, 0x39, - 0x9e, 0x72, 0xc7, 0x92, 0x42, 0x5b, 0x00, 0xc4, 0x75, 0x13, 0xea, 0x12, 0x4e, 0x99, 0x6d, 0xb5, - 0xcd, 0x6e, 0xad, 0xbf, 0x92, 0x7d, 0x6d, 0xc7, 0x75, 0x13, 0x3c, 0xc7, 0xa3, 0xcf, 0x60, 0x23, - 0x26, 0x09, 0xf7, 0x88, 0x2f, 0xbe, 0x22, 0x6b, 0xef, 0x8c, 0x3c, 0x46, 0x06, 0x3e, 0x1d, 0xd9, - 0xc5, 0xb6, 0xd1, 0x2d, 0xe3, 0xfb, 0xda, 0x21, 0xbb, 0x9b, 0x2f, 0x34, 0x8d, 0x7e, 0x5a, 0xb2, - 0x97, 0xf1, 0x84, 0x70, 0xea, 0x4e, 0xec, 0x52, 0xdb, 0xe8, 0xd6, 0xfa, 0x9b, 0xd9, 0x87, 0xbf, - 0x5d, 0x8c, 0x71, 0xa4, 0xdd, 0xfe, 0x17, 0x3c, 0x23, 0xd0, 0x26, 0x54, 0xd9, 0x0b, 0x2f, 0x76, - 0x86, 0xe3, 0x34, 0x7c, 0xc1, 0xec, 0xb2, 0x4c, 0x05, 0x04, 0xb4, 0x27, 0x11, 0xf4, 0x10, 0xac, - 0xb1, 0x17, 0x72, 0x66, 0x57, 0xda, 0x86, 0x2c, 0xa8, 0x52, 0x57, 0x2f, 0x53, 0x57, 0x6f, 0x27, - 0x9c, 0x60, 0xe5, 0x82, 0x10, 0x14, 0x18, 0xa7, 0xb1, 0x0d, 0xb2, 0x6c, 0x72, 0x8d, 0x1a, 0x60, - 0x25, 0x24, 0x74, 0xa9, 0x5d, 0x95, 0xa0, 0x32, 0xd0, 0x63, 0xa8, 0x9e, 0xa6, 0x34, 0x99, 0x38, - 0x2a, 0xf6, 0x8a, 0x8c, 0x8d, 0xb2, 0x53, 0x7c, 0x27, 0xa8, 0x7d, 0xc1, 0x60, 0x38, 0x9d, 0xae, - 0xd1, 0x23, 0x00, 0x36, 0x26, 0xc9, 0xc8, 0xf1, 0xc2, 0x93, 0xc8, 0x5e, 0x95, 0x7b, 0xee, 0x66, - 0x7b, 0x8e, 0x04, 0xf3, 0x34, 0x3c, 0x89, 0x70, 0x85, 0x65, 0x4b, 0xf4, 0x31, 0xac, 0x9f, 0x79, - 0x7c, 0x1c, 0xa5, 0xdc, 0xd1, 0x5a, 0x73, 0x7c, 0x21, 0x04, 0x66, 0xd7, 0xda, 0x66, 0xb7, 0x82, - 0x1b, 0x9a, 0xc5, 0x8a, 0x94, 0x22, 0x61, 0x22, 0x65, 0xdf, 0x0b, 0x3c, 0x6e, 0xdf, 0x51, 0x29, - 0x4b, 0xa3, 0xf3, 0xd2, 0x00, 0x98, 0x25, 0x26, 0x0b, 0xc7, 0x69, 0xec, 0x04, 0x9e, 0xef, 0x7b, - 0x4c, 0x8b, 0x14, 0x04, 0x74, 0x20, 0x11, 0xd4, 0x86, 0xc2, 0x49, 0x1a, 0x0e, 0xa5, 0x46, 0xab, - 0x33, 0x69, 0x3c, 0x49, 0xc3, 0x21, 0x96, 0x0c, 0xda, 0x82, 0xb2, 0x9b, 0x44, 0x69, 0xec, 0x85, - 0xae, 0x54, 0x5a, 0xb5, 0x5f, 0xcf, 0xbc, 0xbe, 0xd2, 0x38, 0x9e, 0x7a, 0xa0, 0x0f, 0xb3, 0x42, - 0x5a, 0xd2, 0x75, 0x35, 0x73, 0xc5, 0x02, 0xd4, 0x75, 0xed, 0x9c, 0x41, 0x65, 0x5a, 0x08, 0x99, - 0xa2, 0xae, 0xd7, 0x88, 0x9e, 0x4f, 0x53, 0x54, 0xfc, 0x88, 0x9e, 0xa3, 0x0f, 0x60, 0x85, 0x47, - 0x9c, 0xf8, 0x8e, 0xc4, 0x98, 0x6e, 0xa7, 0xaa, 0xc4, 0x64, 0x18, 0x86, 0x6a, 0x90, 0x1f, 0x4c, - 0x64, 0xbf, 0x96, 0x71, 0x7e, 0x30, 0x11, 0xcd, 0xad, 0x2b, 0x58, 0x90, 0x15, 0xd4, 0x56, 0xa7, - 0x09, 0x05, 0x71, 0x32, 0x21, 0x81, 0x90, 0xe8, 0xa6, 0xad, 0x60, 0xb9, 0xee, 0xf4, 0xa1, 0x9c, - 0x9d, 0x47, 0xc7, 0x33, 0x96, 0xc4, 0x33, 0x17, 0xe2, 0x6d, 0x82, 0x25, 0x0f, 0x26, 0x1c, 0x16, - 0x4a, 0xac, 0xad, 0xce, 0x6f, 0x06, 0xd4, 0xb2, 0x99, 0xa1, 0x34, 0x8d, 0xba, 0x50, 0x9c, 0xce, - 0x2d, 0x51, 0xa2, 0xda, 0x54, 0x1b, 0x12, 0xdd, 0xcf, 0x61, 0xcd, 0xa3, 0x26, 0x94, 0xce, 0x48, - 0x12, 0x8a, 0xc2, 0xcb, 0x19, 0xb5, 0x9f, 0xc3, 0x19, 0x80, 0xb6, 0x32, 0xc1, 0x9b, 0xaf, 0x17, - 0xfc, 0x7e, 0x4e, 0x4b, 0x7e, 0xb7, 0x0c, 0xc5, 0x84, 0xb2, 0xd4, 0xe7, 0x9d, 0x5f, 0x4d, 0xb8, - 0x2b, 0x05, 0x74, 0x48, 0x82, 0xd9, 0x20, 0x7b, 0x63, 0xe3, 0x1b, 0x37, 0x68, 0xfc, 0xfc, 0x0d, - 0x1b, 0xbf, 0x01, 0x16, 0xe3, 0x24, 0xe1, 0x7a, 0x16, 0x2b, 0x03, 0xd5, 0xc1, 0xa4, 0xe1, 0x48, - 0xcf, 0x3d, 0xb1, 0x9c, 0xf5, 0xbf, 0xf5, 0xf6, 0xfe, 0x9f, 0x9f, 0xbf, 0xc5, 0xf7, 0x98, 0xbf, - 0xaf, 0x6f, 0xd3, 0xd2, 0xbb, 0xb4, 0x69, 0x79, 0xbe, 0x4d, 0x13, 0x40, 0xf3, 0xb7, 0xa0, 0xa5, - 0xd1, 0x00, 0x4b, 0x48, 0x51, 0xfd, 0xa3, 0x55, 0xb0, 0x32, 0x50, 0x13, 0xca, 0xfa, 0xd6, 0x85, - 0xf6, 0x05, 0x31, 0xb5, 0x67, 0xe7, 0x36, 0xdf, 0x7a, 0xee, 0xce, 0x1f, 0xa6, 0xfe, 0xe8, 0xf7, - 0xc4, 0x4f, 0x67, 0x77, 0x2f, 0x12, 0x14, 0xa8, 0x6e, 0x06, 0x65, 0xbc, 0x59, 0x11, 0xf9, 0x1b, - 0x28, 0xc2, 0xbc, 0x2d, 0x45, 0x14, 0x96, 0x28, 0xc2, 0x5a, 0xa2, 0x88, 0xe2, 0xfb, 0x29, 0xa2, - 0x74, 0x2b, 0x8a, 0x28, 0xbf, 0x8b, 0x22, 0x2a, 0xf3, 0x8a, 0x48, 0xe1, 0xde, 0xc2, 0xe5, 0x68, - 0x49, 0xac, 0x43, 0xf1, 0x67, 0x89, 0x68, 0x4d, 0x68, 0xeb, 0xb6, 0x44, 0xf1, 0x70, 0x17, 0x0a, - 0xe2, 0x19, 0x80, 0x4a, 0x60, 0xe2, 0x9d, 0xe3, 0x7a, 0x0e, 0x55, 0xc0, 0xda, 0xfb, 0xe6, 0xf9, - 0xe1, 0xb3, 0xba, 0x21, 0xb0, 0xa3, 0xe7, 0x07, 0xf5, 0xbc, 0x58, 0x1c, 0x3c, 0x3d, 0xac, 0x9b, - 0x72, 0xb1, 0xf3, 0x43, 0xbd, 0x80, 0xaa, 0x50, 0x92, 0x5e, 0x5f, 0xe2, 0xba, 0xd5, 0xff, 0xd3, - 0x00, 0xeb, 0x48, 0xbc, 0xf4, 0xd0, 0xa7, 0x50, 0x54, 0x53, 0x0c, 0xad, 0x2d, 0x4e, 0x35, 0x2d, - 0xb6, 0xe6, 0xfa, 0x75, 0x58, 0x1d, 0xf3, 0x91, 0x81, 0xf6, 0x00, 0x66, 0x1d, 0x81, 0x36, 0x16, - 0xea, 0x3f, 0x3f, 0xab, 0x9a, 0xcd, 0x65, 0x94, 0xae, 0xd6, 0x13, 0xa8, 0xce, 0x15, 0x11, 0x2d, - 0xba, 0x2e, 0xc8, 0xbe, 0xf9, 0x60, 0x29, 0xa7, 0xe2, 0xf4, 0x0f, 0xa1, 0x26, 0xdf, 0x9b, 0x42, - 0xcf, 0xea, 0x64, 0x9f, 0x43, 0x15, 0xd3, 0x20, 0xe2, 0x54, 0xe2, 0x68, 0xaa, 0x8f, 0xf9, 0x67, - 0x69, 0x73, 0xed, 0x1a, 0xaa, 0x9f, 0xaf, 0xb9, 0xdd, 0x8f, 0x2e, 0xfe, 0x6d, 0xe5, 0x2e, 0x2e, - 0x5b, 0xc6, 0xab, 0xcb, 0x96, 0xf1, 0xcf, 0x65, 0xcb, 0xf8, 0xfd, 0xaa, 0x95, 0x7b, 0x75, 0xd5, - 0xca, 0xfd, 0x75, 0xd5, 0xca, 0xfd, 0x58, 0xd2, 0xcf, 0xe4, 0x41, 0x51, 0xde, 0xd0, 0xe3, 0xff, - 0x02, 0x00, 0x00, 0xff, 0xff, 0x84, 0xe1, 0x09, 0x34, 0x90, 0x0b, 0x00, 0x00, + // 1210 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0x4b, 0x6f, 0xdb, 0x46, + 0x10, 0x16, 0x45, 0x51, 0x8f, 0x91, 0xad, 0x28, 0x1b, 0xd9, 0xa1, 0x15, 0x40, 0x56, 0x15, 0x14, + 0x10, 0x02, 0x43, 0x0e, 0x98, 0xa2, 0x40, 0x8b, 0x1e, 0x6a, 0x3b, 0x4d, 0x1d, 0xa0, 0x76, 0x5b, + 0x3a, 0xb1, 0x8b, 0xf6, 0x20, 0x50, 0xd2, 0x9a, 0x22, 0xc2, 0x97, 0xb9, 0xcb, 0xc8, 0x3a, 0xb7, + 0xe7, 0x22, 0xf7, 0xde, 0xfa, 0x6b, 0x7c, 0x6b, 0x8e, 0x3d, 0x15, 0xad, 0xfd, 0x47, 0x8a, 0x7d, + 0x50, 0x22, 0x1d, 0x39, 0x71, 0x60, 0x5f, 0x84, 0x9d, 0xf9, 0x66, 0x66, 0x67, 0x67, 0xbf, 0x19, + 0xae, 0xe0, 0x3e, 0xa1, 0x41, 0x84, 0x37, 0xf9, 0x6f, 0x38, 0xd8, 0x8c, 0xc2, 0x61, 0x2f, 0x8c, + 0x02, 0x1a, 0xa0, 0x22, 0x1d, 0x5b, 0x7e, 0x40, 0x9a, 0x6b, 0x59, 0x03, 0x3a, 0x0d, 0x31, 0x11, + 0x26, 0xcd, 0x86, 0x1d, 0xd8, 0x01, 0x5f, 0x6e, 0xb2, 0x95, 0xd4, 0xb6, 0xb3, 0x0e, 0x61, 0x14, + 0x78, 0x97, 0xfc, 0x1e, 0x66, 0x2d, 0x26, 0x91, 0x43, 0xf1, 0x6b, 0xe3, 0x92, 0xd1, 0x9a, 0x1d, + 0x04, 0xb6, 0x8b, 0x37, 0xb9, 0x34, 0x88, 0x8f, 0x37, 0x2d, 0x7f, 0x2a, 0xa0, 0xce, 0x1d, 0x58, + 0x3e, 0x62, 0x3e, 0x26, 0x26, 0x61, 0xe0, 0x13, 0xdc, 0xf9, 0x55, 0x81, 0x25, 0xa9, 0x39, 0x89, + 0x31, 0xa1, 0x68, 0x0b, 0x80, 0x3a, 0x1e, 0x26, 0x38, 0x72, 0x30, 0xd1, 0x95, 0xb6, 0xda, 0xad, + 0x1a, 0x0f, 0x98, 0xb7, 0x87, 0xe9, 0x18, 0xc7, 0xa4, 0x3f, 0x0c, 0xc2, 0x69, 0xef, 0x85, 0xe3, + 0xe1, 0x03, 0x6e, 0xb2, 0x5d, 0x38, 0xfb, 0x67, 0x3d, 0x67, 0xa6, 0x9c, 0xd0, 0x2a, 0x14, 0x29, + 0xf6, 0x2d, 0x9f, 0xea, 0xf9, 0xb6, 0xd2, 0xad, 0x98, 0x52, 0x42, 0x3a, 0x94, 0x22, 0x1c, 0xba, + 0xce, 0xd0, 0xd2, 0xd5, 0xb6, 0xd2, 0x55, 0xcd, 0x44, 0xec, 0xfc, 0xa9, 0x40, 0x2d, 0x9d, 0xc5, + 0xa1, 0xc1, 0x8c, 0xc9, 0xd4, 0x1b, 0x04, 0xae, 0x48, 0xa2, 0x62, 0x26, 0x22, 0x7a, 0x9a, 0xc9, + 0x30, 0xcf, 0x33, 0x6c, 0xa5, 0x33, 0x94, 0x55, 0xb9, 0x6e, 0x92, 0xea, 0x55, 0x49, 0x16, 0x2e, + 0x25, 0xa9, 0xc1, 0xb2, 0x08, 0x97, 0xd4, 0x6a, 0x0d, 0xca, 0x9e, 0xe3, 0xf7, 0x59, 0x54, 0x5d, + 0x11, 0xc6, 0x9e, 0xe3, 0xb3, 0x6d, 0x39, 0x64, 0x9d, 0x0a, 0x28, 0x2f, 0x21, 0xeb, 0x94, 0x43, + 0x9f, 0x33, 0x88, 0x0e, 0xc7, 0x38, 0x22, 0xba, 0xca, 0xb3, 0x6f, 0xf4, 0x04, 0x63, 0x7a, 0xdf, + 0x59, 0x03, 0xec, 0xee, 0x09, 0x50, 0xe6, 0x3c, 0xb3, 0x45, 0x06, 0xac, 0xb0, 0x90, 0x11, 0x26, + 0x81, 0x1b, 0x53, 0x27, 0xf0, 0xfb, 0x13, 0xc7, 0x1f, 0x05, 0x13, 0x99, 0xe7, 0x3d, 0xcf, 0x3a, + 0x35, 0x67, 0xd8, 0x11, 0x87, 0xd0, 0x06, 0x80, 0x65, 0xdb, 0x11, 0xb6, 0x2d, 0x8a, 0x89, 0xae, + 0xb5, 0xd5, 0x6e, 0xcd, 0x58, 0x4a, 0x76, 0xdb, 0xb2, 0xed, 0xc8, 0x4c, 0xe1, 0xe8, 0x4b, 0x58, + 0x0b, 0xad, 0x88, 0x3a, 0x96, 0xcb, 0x76, 0xe1, 0x04, 0xe9, 0x8f, 0x1c, 0x62, 0x0d, 0x5c, 0x3c, + 0xd2, 0x8b, 0x6d, 0xa5, 0x5b, 0x36, 0xef, 0x4b, 0x83, 0x84, 0x40, 0x4f, 0x25, 0x8c, 0x7e, 0x59, + 0xe0, 0x4b, 0x68, 0x64, 0x51, 0x6c, 0x4f, 0xf5, 0x52, 0x5b, 0xe9, 0xd6, 0x8c, 0xf5, 0x64, 0xe3, + 0x1f, 0xb2, 0x31, 0x0e, 0xa4, 0xd9, 0x3b, 0xc1, 0x13, 0x00, 0xad, 0x43, 0x95, 0xbc, 0x72, 0xc2, + 0xfe, 0x70, 0x1c, 0xfb, 0xaf, 0x88, 0x5e, 0xe6, 0xa9, 0x00, 0x53, 0xed, 0x70, 0x0d, 0x7a, 0x04, + 0xda, 0xd8, 0xf1, 0x29, 0xd1, 0x2b, 0x6d, 0x85, 0x17, 0x54, 0xb4, 0x40, 0x2f, 0x69, 0x81, 0xde, + 0x96, 0x3f, 0x35, 0x85, 0x09, 0x42, 0x50, 0x20, 0x14, 0x87, 0x3a, 0xf0, 0xb2, 0xf1, 0x35, 0x6a, + 0x80, 0x16, 0x59, 0xbe, 0x8d, 0xf5, 0x2a, 0x57, 0x0a, 0x01, 0x3d, 0x81, 0xea, 0x49, 0x8c, 0xa3, + 0x69, 0x5f, 0xc4, 0x5e, 0xe2, 0xb1, 0x51, 0x72, 0x8a, 0x1f, 0x19, 0xb4, 0xcb, 0x10, 0x13, 0x4e, + 0x66, 0x6b, 0xf4, 0x18, 0x80, 0x8c, 0xad, 0x68, 0xd4, 0x77, 0xfc, 0xe3, 0x40, 0x5f, 0xe6, 0x3e, + 0x77, 0x13, 0x9f, 0x03, 0x86, 0x3c, 0xf7, 0x8f, 0x03, 0xb3, 0x42, 0x92, 0x25, 0xfa, 0x0c, 0x56, + 0x27, 0x0e, 0x1d, 0x07, 0x31, 0xed, 0x4b, 0xae, 0xf5, 0x5d, 0x46, 0x04, 0xa2, 0xd7, 0x38, 0xf3, + 0x1b, 0x12, 0x35, 0x05, 0xc8, 0x49, 0x42, 0x58, 0xca, 0xae, 0xe3, 0x39, 0x54, 0xbf, 0x23, 0x52, + 0xe6, 0x02, 0xeb, 0x24, 0x98, 0x27, 0xc6, 0x0b, 0x47, 0x71, 0xd8, 0xf7, 0x1c, 0xd7, 0x75, 0x88, + 0x24, 0x29, 0x30, 0xd5, 0x1e, 0xd7, 0xa0, 0x36, 0x14, 0x8e, 0x63, 0x7f, 0xc8, 0x39, 0x5a, 0x9d, + 0x53, 0xe3, 0x59, 0xec, 0x0f, 0x4d, 0x8e, 0xa0, 0x0d, 0x28, 0xdb, 0x51, 0x10, 0x87, 0x8e, 0x6f, + 0x73, 0xa6, 0x55, 0x8d, 0x7a, 0x62, 0xf5, 0xad, 0xd4, 0x9b, 0x33, 0x0b, 0xf4, 0x30, 0x29, 0xa4, + 0xc6, 0x4d, 0x97, 0x13, 0x53, 0x93, 0x29, 0x65, 0x5d, 0x3b, 0x13, 0xa8, 0xcc, 0x0a, 0xc1, 0x53, + 0x94, 0xf5, 0x1a, 0xe1, 0xd3, 0x59, 0x8a, 0x02, 0x1f, 0xe1, 0x53, 0xf4, 0x09, 0x2c, 0xd1, 0x80, + 0x5a, 0x6e, 0x9f, 0xeb, 0x88, 0x6c, 0xa7, 0x2a, 0xd7, 0xf1, 0x30, 0x04, 0xd5, 0x20, 0x3f, 0x98, + 0xf2, 0x46, 0x2e, 0x9b, 0xf9, 0xc1, 0x94, 0x35, 0xb7, 0xac, 0x60, 0x81, 0x57, 0x50, 0x4a, 0x9d, + 0x26, 0x14, 0xd8, 0xc9, 0x18, 0x05, 0x7c, 0x4b, 0x36, 0x6d, 0xc5, 0xe4, 0xeb, 0x8e, 0x01, 0xe5, + 0xe4, 0x3c, 0x32, 0x9e, 0xb2, 0x20, 0x9e, 0x9a, 0x89, 0xb7, 0x0e, 0x1a, 0x3f, 0x18, 0x33, 0xc8, + 0x94, 0x58, 0x4a, 0x9d, 0xdf, 0x15, 0xa8, 0x25, 0x33, 0x43, 0x70, 0x1a, 0x75, 0xa1, 0x38, 0x1b, + 0xae, 0xac, 0x44, 0xb5, 0x19, 0x37, 0xb8, 0x76, 0x37, 0x67, 0x4a, 0x1c, 0x35, 0xa1, 0x34, 0xb1, + 0x22, 0x9f, 0x15, 0x9e, 0x0f, 0xd2, 0xdd, 0x9c, 0x99, 0x28, 0xd0, 0x46, 0x42, 0x78, 0xf5, 0x6a, + 0xc2, 0xef, 0xe6, 0x24, 0xe5, 0xb7, 0xcb, 0x50, 0x8c, 0x30, 0x89, 0x5d, 0xda, 0xf9, 0x4d, 0x85, + 0xbb, 0x9c, 0x40, 0xfb, 0x96, 0x37, 0x1f, 0x64, 0xef, 0x6d, 0x7c, 0xe5, 0x06, 0x8d, 0x9f, 0xbf, + 0x61, 0xe3, 0x37, 0x40, 0x23, 0xd4, 0x8a, 0xa8, 0xfc, 0x60, 0x08, 0x01, 0xd5, 0x41, 0xc5, 0xfe, + 0x48, 0xce, 0x3d, 0xb6, 0x9c, 0xf7, 0xbf, 0xf6, 0xe1, 0xfe, 0x4f, 0xcf, 0xdf, 0xe2, 0x47, 0xcc, + 0xdf, 0xab, 0xdb, 0xb4, 0x74, 0x9d, 0x36, 0x2d, 0xa7, 0xdb, 0x34, 0x02, 0x94, 0xbe, 0x05, 0x49, + 0x8d, 0x06, 0x68, 0x8c, 0x8a, 0xc9, 0x17, 0x4f, 0x08, 0xa8, 0x09, 0x65, 0x79, 0xeb, 0xe2, 0x6b, + 0x57, 0x31, 0x67, 0xf2, 0xfc, 0xdc, 0xea, 0x07, 0xcf, 0xdd, 0xf9, 0x43, 0x95, 0x9b, 0x1e, 0x5a, + 0x6e, 0x3c, 0xbf, 0x7b, 0x96, 0x20, 0xd3, 0xca, 0x66, 0x10, 0xc2, 0xfb, 0x19, 0x91, 0xbf, 0x01, + 0x23, 0xd4, 0xdb, 0x62, 0x44, 0x61, 0x01, 0x23, 0xb4, 0x05, 0x8c, 0x28, 0x7e, 0x1c, 0x23, 0x4a, + 0xb7, 0xc2, 0x88, 0xf2, 0x75, 0x18, 0x51, 0x49, 0x33, 0x22, 0x86, 0x7b, 0x99, 0xcb, 0x91, 0x94, + 0x58, 0x85, 0xe2, 0x6b, 0xae, 0x91, 0x9c, 0x90, 0xd2, 0x6d, 0x91, 0xe2, 0xd1, 0x36, 0x14, 0xd8, + 0x33, 0x00, 0x95, 0x40, 0x35, 0xb7, 0x8e, 0xea, 0x39, 0x54, 0x01, 0x6d, 0xe7, 0xfb, 0x97, 0xfb, + 0x2f, 0xea, 0x0a, 0xd3, 0x1d, 0xbc, 0xdc, 0xab, 0xe7, 0xd9, 0x62, 0xef, 0xf9, 0x7e, 0x5d, 0xe5, + 0x8b, 0xad, 0x9f, 0xea, 0x05, 0x54, 0x85, 0x12, 0xb7, 0xfa, 0xc6, 0xac, 0x6b, 0xc6, 0x5f, 0x0a, + 0x68, 0x07, 0xec, 0x45, 0x8a, 0xbe, 0x80, 0xa2, 0x98, 0x62, 0x68, 0x25, 0x3b, 0xd5, 0x24, 0xd9, + 0x9a, 0xab, 0x97, 0xd5, 0xe2, 0x98, 0x8f, 0x15, 0xb4, 0x03, 0x30, 0xef, 0x08, 0xb4, 0x96, 0xa9, + 0x7f, 0x7a, 0x56, 0x35, 0x9b, 0x8b, 0x20, 0x59, 0xad, 0x67, 0x50, 0x4d, 0x15, 0x11, 0x65, 0x4d, + 0x33, 0xb4, 0x6f, 0x3e, 0x58, 0x88, 0x89, 0x38, 0xc6, 0x9b, 0xe4, 0x3d, 0xca, 0x08, 0x2d, 0x8e, + 0xf6, 0x15, 0x54, 0x4d, 0xec, 0x05, 0x14, 0x73, 0x3d, 0x9a, 0x11, 0x24, 0xfd, 0x6c, 0x6d, 0xae, + 0x5c, 0xd2, 0xca, 0x47, 0x76, 0x0e, 0x7d, 0x0d, 0xcb, 0x29, 0xef, 0x43, 0x03, 0xad, 0x2e, 0xf2, + 0x3f, 0x34, 0xae, 0x8c, 0xb0, 0xfd, 0xe9, 0xd9, 0x7f, 0xad, 0xdc, 0xd9, 0x79, 0x4b, 0x79, 0x7b, + 0xde, 0x52, 0xfe, 0x3d, 0x6f, 0x29, 0x6f, 0x2e, 0x5a, 0xb9, 0xb7, 0x17, 0xad, 0xdc, 0xdf, 0x17, + 0xad, 0xdc, 0xcf, 0x25, 0xf9, 0x8f, 0x60, 0x50, 0xe4, 0x97, 0xfc, 0xe4, 0xff, 0x00, 0x00, 0x00, + 0xff, 0xff, 0x54, 0x90, 0x80, 0x78, 0x9d, 0x0c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -1017,6 +1063,7 @@ var _Store_serviceDesc = grpc.ServiceDesc{ type WriteableStoreClient interface { // WriteRequest allows you to write metrics to this store via remote write RemoteWrite(ctx context.Context, in *WriteRequest, opts ...grpc.CallOption) (*WriteResponse, error) + RemoteWriteV2(ctx context.Context, in *WriteRequestV2, opts ...grpc.CallOption) (*WriteResponse, error) } type writeableStoreClient struct { @@ -1036,10 +1083,20 @@ func (c *writeableStoreClient) RemoteWrite(ctx context.Context, in *WriteRequest return out, nil } +func (c *writeableStoreClient) RemoteWriteV2(ctx context.Context, in *WriteRequestV2, opts ...grpc.CallOption) (*WriteResponse, error) { + out := new(WriteResponse) + err := c.cc.Invoke(ctx, "/thanos.WriteableStore/RemoteWriteV2", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // WriteableStoreServer is the server API for WriteableStore service. type WriteableStoreServer interface { // WriteRequest allows you to write metrics to this store via remote write RemoteWrite(context.Context, *WriteRequest) (*WriteResponse, error) + RemoteWriteV2(context.Context, *WriteRequestV2) (*WriteResponse, error) } // UnimplementedWriteableStoreServer can be embedded to have forward compatible implementations. @@ -1049,6 +1106,9 @@ type UnimplementedWriteableStoreServer struct { func (*UnimplementedWriteableStoreServer) RemoteWrite(ctx context.Context, req *WriteRequest) (*WriteResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method RemoteWrite not implemented") } +func (*UnimplementedWriteableStoreServer) RemoteWriteV2(ctx context.Context, req *WriteRequestV2) (*WriteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RemoteWriteV2 not implemented") +} func RegisterWriteableStoreServer(s *grpc.Server, srv WriteableStoreServer) { s.RegisterService(&_WriteableStore_serviceDesc, srv) @@ -1072,6 +1132,24 @@ func _WriteableStore_RemoteWrite_Handler(srv interface{}, ctx context.Context, d return interceptor(ctx, in, info, handler) } +func _WriteableStore_RemoteWriteV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(WriteRequestV2) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WriteableStoreServer).RemoteWriteV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/thanos.WriteableStore/RemoteWriteV2", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WriteableStoreServer).RemoteWriteV2(ctx, req.(*WriteRequestV2)) + } + return interceptor(ctx, in, info, handler) +} + var _WriteableStore_serviceDesc = grpc.ServiceDesc{ ServiceName: "thanos.WriteableStore", HandlerType: (*WriteableStoreServer)(nil), @@ -1080,6 +1158,10 @@ var _WriteableStore_serviceDesc = grpc.ServiceDesc{ MethodName: "RemoteWrite", Handler: _WriteableStore_RemoteWrite_Handler, }, + { + MethodName: "RemoteWriteV2", + Handler: _WriteableStore_RemoteWriteV2_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "store/storepb/rpc.proto", @@ -1157,6 +1239,64 @@ func (m *WriteRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *WriteRequestV2) 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 *WriteRequestV2) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *WriteRequestV2) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Replica != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.Replica)) + i-- + dAtA[i] = 0x20 + } + if len(m.Tenant) > 0 { + i -= len(m.Tenant) + copy(dAtA[i:], m.Tenant) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Tenant))) + i-- + dAtA[i] = 0x1a + } + if len(m.Timeseries) > 0 { + for iNdEx := len(m.Timeseries) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Timeseries[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.Symbols) > 0 { + for iNdEx := len(m.Symbols) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Symbols[iNdEx]) + copy(dAtA[i:], m.Symbols[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Symbols[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func (m *SeriesRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1947,6 +2087,34 @@ func (m *WriteRequest) Size() (n int) { return n } +func (m *WriteRequestV2) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Symbols) > 0 { + for _, s := range m.Symbols { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + if len(m.Timeseries) > 0 { + for _, e := range m.Timeseries { + l = e.Size() + n += 1 + l + sovRpc(uint64(l)) + } + } + l = len(m.Tenant) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + if m.Replica != 0 { + n += 1 + sovRpc(uint64(m.Replica)) + } + return n +} + func (m *SeriesRequest) Size() (n int) { if m == nil { return 0 @@ -2476,6 +2644,173 @@ func (m *WriteRequest) Unmarshal(dAtA []byte) error { } return nil } +func (m *WriteRequestV2) 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 ErrIntOverflowRpc + } + 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: WriteRequestV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WriteRequestV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Symbols", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Symbols = append(m.Symbols, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timeseries", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Timeseries = append(m.Timeseries, writev2pb.TimeSeries{}) + if err := m.Timeseries[len(m.Timeseries)-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 Tenant", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Tenant = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Replica", wireType) + } + m.Replica = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Replica |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *SeriesRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/store/storepb/rpc.proto b/pkg/store/storepb/rpc.proto index 2de086bc12..52a088dafc 100644 --- a/pkg/store/storepb/rpc.proto +++ b/pkg/store/storepb/rpc.proto @@ -7,6 +7,7 @@ package thanos; import "store/storepb/types.proto"; import "gogoproto/gogo.proto"; import "store/storepb/prompb/types.proto"; +import "store/storepb/writev2pb/types.proto"; import "google/protobuf/any.proto"; option go_package = "storepb"; @@ -49,6 +50,8 @@ service Store { service WriteableStore { // WriteRequest allows you to write metrics to this store via remote write rpc RemoteWrite(WriteRequest) returns (WriteResponse) {} + + rpc RemoteWriteV2(WriteRequestV2) returns (WriteResponse) {} } message WriteResponse { @@ -60,6 +63,13 @@ message WriteRequest { int64 replica = 3; } +message WriteRequestV2 { + repeated string symbols = 1; + repeated prometheus_writev2.TimeSeries timeseries = 2 [(gogoproto.nullable) = false]; + string tenant = 3; + int64 replica = 4; +} + message SeriesRequest { int64 min_time = 1; int64 max_time = 2; diff --git a/pkg/store/storepb/writev2pb/codec.go b/pkg/store/storepb/writev2pb/codec.go new file mode 100644 index 0000000000..3c13055f19 --- /dev/null +++ b/pkg/store/storepb/writev2pb/codec.go @@ -0,0 +1,217 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +// Copyright 2024 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package writev2pb + +import ( + "github.com/prometheus/common/model" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" +) + +// ToLabels return model labels.Labels from timeseries' remote labels. +func (m TimeSeries) ToLabels(b *labels.ScratchBuilder, symbols []string) labels.Labels { + return DesymbolizeLabels(b, m.GetLabelsRefs(), symbols) +} + +// ToMetadata return model metadata from timeseries' remote metadata. +func (m TimeSeries) ToMetadata(symbols []string) metadata.Metadata { + typ := model.MetricTypeUnknown + switch m.Metadata.Type { + case Metadata_METRIC_TYPE_COUNTER: + typ = model.MetricTypeCounter + case Metadata_METRIC_TYPE_GAUGE: + typ = model.MetricTypeGauge + case Metadata_METRIC_TYPE_HISTOGRAM: + typ = model.MetricTypeHistogram + case Metadata_METRIC_TYPE_GAUGEHISTOGRAM: + typ = model.MetricTypeGaugeHistogram + case Metadata_METRIC_TYPE_SUMMARY: + typ = model.MetricTypeSummary + case Metadata_METRIC_TYPE_INFO: + typ = model.MetricTypeInfo + case Metadata_METRIC_TYPE_STATESET: + typ = model.MetricTypeStateset + } + return metadata.Metadata{ + Type: typ, + Unit: symbols[m.Metadata.UnitRef], + Help: symbols[m.Metadata.HelpRef], + } +} + +// FromMetadataType transforms a Prometheus metricType into writev2 metricType. +// Since the former is a string we need to transform it to an enum. +func FromMetadataType(t model.MetricType) Metadata_MetricType { + switch t { + case model.MetricTypeCounter: + return Metadata_METRIC_TYPE_COUNTER + case model.MetricTypeGauge: + return Metadata_METRIC_TYPE_GAUGE + case model.MetricTypeHistogram: + return Metadata_METRIC_TYPE_HISTOGRAM + case model.MetricTypeGaugeHistogram: + return Metadata_METRIC_TYPE_GAUGEHISTOGRAM + case model.MetricTypeSummary: + return Metadata_METRIC_TYPE_SUMMARY + case model.MetricTypeInfo: + return Metadata_METRIC_TYPE_INFO + case model.MetricTypeStateset: + return Metadata_METRIC_TYPE_STATESET + default: + return Metadata_METRIC_TYPE_UNSPECIFIED + } +} + +// IsFloatHistogram returns true if the histogram is float. +func (h Histogram) IsFloatHistogram() bool { + _, ok := h.GetCount().(*Histogram_CountFloat) + return ok +} + +// ToIntHistogram returns integer Prometheus histogram from the remote implementation +// of integer histogram. If it's a float histogram, the method returns nil. +func (h Histogram) ToIntHistogram() *histogram.Histogram { + if h.IsFloatHistogram() { + return nil + } + return &histogram.Histogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: h.GetZeroCountInt(), + Count: h.GetCountInt(), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: h.GetPositiveDeltas(), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: h.GetNegativeDeltas(), + CustomValues: h.GetCustomValues(), + } +} + +// ToFloatHistogram returns float Prometheus histogram from the remote implementation +// of float histogram. If the underlying implementation is an integer histogram, a +// conversion is performed. +func (h Histogram) ToFloatHistogram() *histogram.FloatHistogram { + if h.IsFloatHistogram() { + return &histogram.FloatHistogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: h.GetZeroCountFloat(), + Count: h.GetCountFloat(), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: h.GetPositiveCounts(), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: h.GetNegativeCounts(), + CustomValues: h.GetCustomValues(), + } + } + // Conversion from integer histogram. + return &histogram.FloatHistogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: float64(h.GetZeroCountInt()), + Count: float64(h.GetCountInt()), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: deltasToCounts(h.GetPositiveDeltas()), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: deltasToCounts(h.GetNegativeDeltas()), + CustomValues: h.GetCustomValues(), + } +} + +func spansProtoToSpans(s []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 +} + +func deltasToCounts(deltas []int64) []float64 { + counts := make([]float64, len(deltas)) + var cur float64 + for i, d := range deltas { + cur += float64(d) + counts[i] = cur + } + return counts +} + +// FromIntHistogram returns remote Histogram from the integer Histogram. +func FromIntHistogram(timestamp int64, h *histogram.Histogram) Histogram { + return Histogram{ + Count: &Histogram_CountInt{CountInt: h.Count}, + Sum: h.Sum, + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: &Histogram_ZeroCountInt{ZeroCountInt: h.ZeroCount}, + NegativeSpans: spansToSpansProto(h.NegativeSpans), + NegativeDeltas: h.NegativeBuckets, + PositiveSpans: spansToSpansProto(h.PositiveSpans), + PositiveDeltas: h.PositiveBuckets, + ResetHint: Histogram_ResetHint(h.CounterResetHint), + CustomValues: h.CustomValues, + Timestamp: timestamp, + } +} + +// FromFloatHistogram returns remote Histogram from the float Histogram. +func FromFloatHistogram(timestamp int64, fh *histogram.FloatHistogram) Histogram { + return Histogram{ + Count: &Histogram_CountFloat{CountFloat: fh.Count}, + Sum: fh.Sum, + Schema: fh.Schema, + ZeroThreshold: fh.ZeroThreshold, + ZeroCount: &Histogram_ZeroCountFloat{ZeroCountFloat: fh.ZeroCount}, + NegativeSpans: spansToSpansProto(fh.NegativeSpans), + NegativeCounts: fh.NegativeBuckets, + PositiveSpans: spansToSpansProto(fh.PositiveSpans), + PositiveCounts: fh.PositiveBuckets, + ResetHint: Histogram_ResetHint(fh.CounterResetHint), + CustomValues: fh.CustomValues, + Timestamp: timestamp, + } +} + +func spansToSpansProto(s []histogram.Span) []BucketSpan { + spans := make([]BucketSpan, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = BucketSpan{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} + +func (m Exemplar) ToExemplar(b *labels.ScratchBuilder, symbols []string) exemplar.Exemplar { + timestamp := m.Timestamp + + return exemplar.Exemplar{ + Labels: DesymbolizeLabels(b, m.LabelsRefs, symbols), + Value: m.Value, + Ts: timestamp, + HasTs: timestamp != 0, + } +} diff --git a/pkg/store/storepb/writev2pb/codec_test.go b/pkg/store/storepb/writev2pb/codec_test.go new file mode 100644 index 0000000000..884b0ea16e --- /dev/null +++ b/pkg/store/storepb/writev2pb/codec_test.go @@ -0,0 +1,300 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +// Copyright 2024 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package writev2pb + +import ( + "testing" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/prompb" +) + +func TestToLabels(t *testing.T) { + expected := labels.FromStrings("__name__", "metric1", "foo", "bar") + + t.Run("v1", func(t *testing.T) { + ts := prompb.TimeSeries{Labels: []prompb.Label{{Name: "__name__", Value: "metric1"}, {Name: "foo", Value: "bar"}}} + b := labels.NewScratchBuilder(2) + require.Equal(t, expected, ts.ToLabels(&b, nil)) + require.Equal(t, ts.Labels, prompb.FromLabels(expected, nil)) + require.Equal(t, ts.Labels, prompb.FromLabels(expected, ts.Labels)) + }) + t.Run("v2", func(t *testing.T) { + v2Symbols := []string{"", "__name__", "metric1", "foo", "bar"} + ts := TimeSeries{LabelsRefs: []uint32{1, 2, 3, 4}} + b := labels.NewScratchBuilder(2) + require.Equal(t, expected, ts.ToLabels(&b, v2Symbols)) + // No need for FromLabels in our prod code as we use symbol table to do so. + }) +} + +func TestFromMetadataType(t *testing.T) { + for _, tc := range []struct { + desc string + input model.MetricType + expectedV1 prompb.MetricMetadata_MetricType + expectedV2 Metadata_MetricType + }{ + { + desc: "with a single-word metric", + input: model.MetricTypeCounter, + expectedV1: prompb.MetricMetadata_COUNTER, + expectedV2: Metadata_METRIC_TYPE_COUNTER, + }, + { + desc: "with a two-word metric", + input: model.MetricTypeStateset, + expectedV1: prompb.MetricMetadata_STATESET, + expectedV2: Metadata_METRIC_TYPE_STATESET, + }, + { + desc: "with an unknown metric", + input: "not-known", + expectedV1: prompb.MetricMetadata_UNKNOWN, + expectedV2: Metadata_METRIC_TYPE_UNSPECIFIED, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + t.Run("v1", func(t *testing.T) { + require.Equal(t, tc.expectedV1, prompb.FromMetadataType(tc.input)) + }) + t.Run("v2", func(t *testing.T) { + require.Equal(t, tc.expectedV2, FromMetadataType(tc.input)) + }) + }) + } +} + +func TestToMetadata(t *testing.T) { + sym := NewSymbolTable() + + for _, tc := range []struct { + input Metadata + expected metadata.Metadata + }{ + { + input: Metadata{}, + expected: metadata.Metadata{ + Type: model.MetricTypeUnknown, + }, + }, + { + input: Metadata{ + Type: 12414, // Unknown. + }, + expected: metadata.Metadata{ + Type: model.MetricTypeUnknown, + }, + }, + { + input: Metadata{ + Type: Metadata_METRIC_TYPE_COUNTER, + HelpRef: sym.Symbolize("help1"), + UnitRef: sym.Symbolize("unit1"), + }, + expected: metadata.Metadata{ + Type: model.MetricTypeCounter, + Help: "help1", + Unit: "unit1", + }, + }, + { + input: Metadata{ + Type: Metadata_METRIC_TYPE_STATESET, + HelpRef: sym.Symbolize("help2"), + }, + expected: metadata.Metadata{ + Type: model.MetricTypeStateset, + Help: "help2", + }, + }, + } { + t.Run("", func(t *testing.T) { + ts := TimeSeries{Metadata: tc.input} + require.Equal(t, tc.expected, ts.ToMetadata(sym.Symbols())) + }) + } +} + +func TestToHistogram_Empty(t *testing.T) { + t.Run("v1", func(t *testing.T) { + require.NotNilf(t, prompb.Histogram{}.ToIntHistogram(), "") + require.NotNilf(t, prompb.Histogram{}.ToFloatHistogram(), "") + }) + t.Run("v2", func(t *testing.T) { + require.NotNilf(t, Histogram{}.ToIntHistogram(), "") + require.NotNilf(t, Histogram{}.ToFloatHistogram(), "") + }) +} + +// NOTE(bwplotka): This is technically not a valid histogram, but it represents +// important cases to test when copying or converting to/from int/float histograms. +func testIntHistogram() histogram.Histogram { + return histogram.Histogram{ + CounterResetHint: histogram.GaugeType, + Schema: 1, + Count: 19, + Sum: 2.7, + ZeroThreshold: 1e-128, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 4}, + {Offset: 0, Length: 0}, + {Offset: 0, Length: 3}, + }, + PositiveBuckets: []int64{1, 2, -2, 1, -1, 0, 0}, + NegativeSpans: []histogram.Span{ + {Offset: 0, Length: 5}, + {Offset: 1, Length: 0}, + {Offset: 0, Length: 1}, + }, + NegativeBuckets: []int64{1, 2, -2, 1, -1, 0}, + CustomValues: []float64{21421, 523}, + } +} + +// NOTE(bwplotka): This is technically not a valid histogram, but it represents +// important cases to test when copying or converting to/from int/float histograms. +func testFloatHistogram() histogram.FloatHistogram { + return histogram.FloatHistogram{ + CounterResetHint: histogram.GaugeType, + Schema: 1, + Count: 19, + Sum: 2.7, + ZeroThreshold: 1e-128, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 4}, + {Offset: 0, Length: 0}, + {Offset: 0, Length: 3}, + }, + PositiveBuckets: []float64{1, 3, 1, 2, 1, 1, 1}, + NegativeSpans: []histogram.Span{ + {Offset: 0, Length: 5}, + {Offset: 1, Length: 0}, + {Offset: 0, Length: 1}, + }, + NegativeBuckets: []float64{1, 3, 1, 2, 1, 1}, + CustomValues: []float64{21421, 523}, + } +} + +func TestFromIntToFloatOrIntHistogram(t *testing.T) { + t.Run("v1", func(t *testing.T) { + // v1 does not support nhcb. + testIntHistWithoutNHCB := testIntHistogram() + testIntHistWithoutNHCB.CustomValues = nil + testFloatHistWithoutNHCB := testFloatHistogram() + testFloatHistWithoutNHCB.CustomValues = nil + + h := prompb.FromIntHistogram(123, &testIntHistWithoutNHCB) + require.False(t, h.IsFloatHistogram()) + require.Equal(t, int64(123), h.Timestamp) + require.Equal(t, testIntHistWithoutNHCB, *h.ToIntHistogram()) + require.Equal(t, testFloatHistWithoutNHCB, *h.ToFloatHistogram()) + }) + t.Run("v2", func(t *testing.T) { + testIntHist := testIntHistogram() + testFloatHist := testFloatHistogram() + + h := FromIntHistogram(123, &testIntHist) + require.False(t, h.IsFloatHistogram()) + require.Equal(t, int64(123), h.Timestamp) + require.Equal(t, testIntHist, *h.ToIntHistogram()) + require.Equal(t, testFloatHist, *h.ToFloatHistogram()) + }) +} + +func TestFromFloatToFloatHistogram(t *testing.T) { + t.Run("v1", func(t *testing.T) { + // v1 does not support nhcb. + testFloatHistWithoutNHCB := testFloatHistogram() + testFloatHistWithoutNHCB.CustomValues = nil + + h := prompb.FromFloatHistogram(123, &testFloatHistWithoutNHCB) + require.True(t, h.IsFloatHistogram()) + require.Equal(t, int64(123), h.Timestamp) + require.Nil(t, h.ToIntHistogram()) + require.Equal(t, testFloatHistWithoutNHCB, *h.ToFloatHistogram()) + }) + t.Run("v2", func(t *testing.T) { + testFloatHist := testFloatHistogram() + + h := FromFloatHistogram(123, &testFloatHist) + require.True(t, h.IsFloatHistogram()) + require.Equal(t, int64(123), h.Timestamp) + require.Nil(t, h.ToIntHistogram()) + require.Equal(t, testFloatHist, *h.ToFloatHistogram()) + }) +} + +func TestFromIntOrFloatHistogram_ResetHint(t *testing.T) { + for _, tc := range []struct { + input histogram.CounterResetHint + expectedV1 prompb.Histogram_ResetHint + expectedV2 Histogram_ResetHint + }{ + { + input: histogram.UnknownCounterReset, + expectedV1: prompb.Histogram_UNKNOWN, + expectedV2: Histogram_RESET_HINT_UNSPECIFIED, + }, + { + input: histogram.CounterReset, + expectedV1: prompb.Histogram_YES, + expectedV2: Histogram_RESET_HINT_YES, + }, + { + input: histogram.NotCounterReset, + expectedV1: prompb.Histogram_NO, + expectedV2: Histogram_RESET_HINT_NO, + }, + { + input: histogram.GaugeType, + expectedV1: prompb.Histogram_GAUGE, + expectedV2: Histogram_RESET_HINT_GAUGE, + }, + } { + t.Run("", func(t *testing.T) { + t.Run("v1", func(t *testing.T) { + h := testIntHistogram() + h.CounterResetHint = tc.input + got := prompb.FromIntHistogram(1337, &h) + require.Equal(t, tc.expectedV1, got.GetResetHint()) + + fh := testFloatHistogram() + fh.CounterResetHint = tc.input + got2 := prompb.FromFloatHistogram(1337, &fh) + require.Equal(t, tc.expectedV1, got2.GetResetHint()) + }) + t.Run("v2", func(t *testing.T) { + h := testIntHistogram() + h.CounterResetHint = tc.input + got := FromIntHistogram(1337, &h) + require.Equal(t, tc.expectedV2, got.GetResetHint()) + + fh := testFloatHistogram() + fh.CounterResetHint = tc.input + got2 := FromFloatHistogram(1337, &fh) + require.Equal(t, tc.expectedV2, got2.GetResetHint()) + }) + }) + } +} diff --git a/pkg/store/storepb/writev2pb/symbols.go b/pkg/store/storepb/writev2pb/symbols.go new file mode 100644 index 0000000000..5c967ec1bd --- /dev/null +++ b/pkg/store/storepb/writev2pb/symbols.go @@ -0,0 +1,96 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package writev2pb + +import ( + "github.com/prometheus/prometheus/model/labels" +) + +// SymbolsTable implements table for easy symbol use. +type SymbolsTable struct { + strings []string + symbolsMap map[string]uint32 +} + +// NewSymbolTable returns a symbol table. +// The first element of the symbols table is always an empty string so index will be 1-based. +func NewSymbolTable() *SymbolsTable { + return &SymbolsTable{ + // Empty string is required as a first element. + symbolsMap: map[string]uint32{"": 0}, + strings: []string{""}, + } +} + +// NewSymbolTableFromSymbols returns a symbol table from a list of symbols. +func NewSymbolTableFromSymbols(symbols []string) *SymbolsTable { + st := NewSymbolTable() + for _, symbol := range symbols { + st.Symbolize(symbol) + } + return st +} + +// Symbolize adds (if not added before) a string to the symbols table, +// while returning its reference number. +func (t *SymbolsTable) Symbolize(str string) uint32 { + if ref, ok := t.symbolsMap[str]; ok { + return ref + } + ref := uint32(len(t.strings)) + t.strings = append(t.strings, str) + t.symbolsMap[str] = ref + return ref +} + +// SymbolizeLabels symbolize Prometheus labels. +func (t *SymbolsTable) SymbolizeLabels(lbls labels.Labels, buf []uint32) []uint32 { + buf = buf[:0] + if cap(buf) < lbls.Len()*2 { + buf = make([]uint32, 0, lbls.Len()*2) + } + + lbls.Range(func(l labels.Label) { + buf = append(buf, + t.Symbolize(l.Name), + t.Symbolize(l.Value)) + }) + + return buf +} + +// SymbolizeMetadata symbolizes metadata help and unit text. +func (t *SymbolsTable) SymbolizeMetadata(help, unit string) (uint32, uint32) { + return t.Symbolize(help), t.Symbolize(unit) +} + +// Symbols returns computes symbols table to put in e.g. Request.Symbols. +// As per spec, order does not matter. +func (t *SymbolsTable) Symbols() []string { + return t.strings +} + +// Reset clears symbols table. +func (t *SymbolsTable) Reset() { + // NOTE: Make sure to keep empty symbol. + t.strings = t.strings[:1] + for k := range t.symbolsMap { + if k == "" { + continue + } + delete(t.symbolsMap, k) + } +} + +// desymbolizeLabels decodes label references, with given symbols to labels. +func DesymbolizeLabels(b *labels.ScratchBuilder, labelRefs []uint32, symbols []string) labels.Labels { + b.Reset() + for i := 0; i < len(labelRefs); i += 2 { + name := symbols[labelRefs[i]] + value := symbols[labelRefs[i+1]] + b.Add(name, value) + } + b.Sort() + return b.Labels() +} diff --git a/pkg/store/storepb/writev2pb/symbols_test.go b/pkg/store/storepb/writev2pb/symbols_test.go new file mode 100644 index 0000000000..097fd44601 --- /dev/null +++ b/pkg/store/storepb/writev2pb/symbols_test.go @@ -0,0 +1,63 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +// Copyright 2024 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package writev2pb + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/model/labels" +) + +func TestSymbolsTable(t *testing.T) { + s := NewSymbolTable() + require.Equal(t, []string{""}, s.Symbols(), "required empty reference does not exist") + require.Equal(t, uint32(0), s.Symbolize("")) + require.Equal(t, []string{""}, s.Symbols()) + + require.Equal(t, uint32(1), s.Symbolize("abc")) + require.Equal(t, []string{"", "abc"}, s.Symbols()) + + require.Equal(t, uint32(2), s.Symbolize("__name__")) + require.Equal(t, []string{"", "abc", "__name__"}, s.Symbols()) + + require.Equal(t, uint32(3), s.Symbolize("foo")) + require.Equal(t, []string{"", "abc", "__name__", "foo"}, s.Symbols()) + + s.Reset() + require.Equal(t, []string{""}, s.Symbols(), "required empty reference does not exist") + require.Equal(t, uint32(0), s.Symbolize("")) + + require.Equal(t, uint32(1), s.Symbolize("__name__")) + require.Equal(t, []string{"", "__name__"}, s.Symbols()) + + require.Equal(t, uint32(2), s.Symbolize("abc")) + require.Equal(t, []string{"", "__name__", "abc"}, s.Symbols()) + + ls := labels.FromStrings("__name__", "qwer", "zxcv", "1234") + encoded := s.SymbolizeLabels(ls, nil) + require.Equal(t, []uint32{1, 3, 4, 5}, encoded) + b := labels.NewScratchBuilder(len(encoded)) + decoded := DesymbolizeLabels(&b, encoded, s.Symbols()) + require.Equal(t, ls, decoded) + + // Different buf. + ls = labels.FromStrings("__name__", "qwer", "zxcv2222", "1234") + encoded = s.SymbolizeLabels(ls, []uint32{1, 3, 4, 5}) + require.Equal(t, []uint32{1, 3, 6, 5}, encoded) +} diff --git a/pkg/store/storepb/writev2pb/types.pb.go b/pkg/store/storepb/writev2pb/types.pb.go new file mode 100644 index 0000000000..0171619532 --- /dev/null +++ b/pkg/store/storepb/writev2pb/types.pb.go @@ -0,0 +1,3161 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: store/storepb/writev2pb/types.proto + +package writev2pb + +import ( + encoding_binary "encoding/binary" + fmt "fmt" + io "io" + math "math" + math_bits "math/bits" + + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +type Metadata_MetricType int32 + +const ( + Metadata_METRIC_TYPE_UNSPECIFIED Metadata_MetricType = 0 + Metadata_METRIC_TYPE_COUNTER Metadata_MetricType = 1 + Metadata_METRIC_TYPE_GAUGE Metadata_MetricType = 2 + Metadata_METRIC_TYPE_HISTOGRAM Metadata_MetricType = 3 + Metadata_METRIC_TYPE_GAUGEHISTOGRAM Metadata_MetricType = 4 + Metadata_METRIC_TYPE_SUMMARY Metadata_MetricType = 5 + Metadata_METRIC_TYPE_INFO Metadata_MetricType = 6 + Metadata_METRIC_TYPE_STATESET Metadata_MetricType = 7 +) + +var Metadata_MetricType_name = map[int32]string{ + 0: "METRIC_TYPE_UNSPECIFIED", + 1: "METRIC_TYPE_COUNTER", + 2: "METRIC_TYPE_GAUGE", + 3: "METRIC_TYPE_HISTOGRAM", + 4: "METRIC_TYPE_GAUGEHISTOGRAM", + 5: "METRIC_TYPE_SUMMARY", + 6: "METRIC_TYPE_INFO", + 7: "METRIC_TYPE_STATESET", +} + +var Metadata_MetricType_value = map[string]int32{ + "METRIC_TYPE_UNSPECIFIED": 0, + "METRIC_TYPE_COUNTER": 1, + "METRIC_TYPE_GAUGE": 2, + "METRIC_TYPE_HISTOGRAM": 3, + "METRIC_TYPE_GAUGEHISTOGRAM": 4, + "METRIC_TYPE_SUMMARY": 5, + "METRIC_TYPE_INFO": 6, + "METRIC_TYPE_STATESET": 7, +} + +func (x Metadata_MetricType) String() string { + return proto.EnumName(Metadata_MetricType_name, int32(x)) +} + +func (Metadata_MetricType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_9c40b9525775fffe, []int{4, 0} +} + +type Histogram_ResetHint int32 + +const ( + Histogram_RESET_HINT_UNSPECIFIED Histogram_ResetHint = 0 + Histogram_RESET_HINT_YES Histogram_ResetHint = 1 + Histogram_RESET_HINT_NO Histogram_ResetHint = 2 + Histogram_RESET_HINT_GAUGE Histogram_ResetHint = 3 +) + +var Histogram_ResetHint_name = map[int32]string{ + 0: "RESET_HINT_UNSPECIFIED", + 1: "RESET_HINT_YES", + 2: "RESET_HINT_NO", + 3: "RESET_HINT_GAUGE", +} + +var Histogram_ResetHint_value = map[string]int32{ + "RESET_HINT_UNSPECIFIED": 0, + "RESET_HINT_YES": 1, + "RESET_HINT_NO": 2, + "RESET_HINT_GAUGE": 3, +} + +func (x Histogram_ResetHint) String() string { + return proto.EnumName(Histogram_ResetHint_name, int32(x)) +} + +func (Histogram_ResetHint) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_9c40b9525775fffe, []int{5, 0} +} + +// Request represents a request to write the given timeseries to a remote destination. +// This message was introduced in the Remote Write 2.0 specification: +// https://prometheus.io/docs/concepts/remote_write_spec_2_0/ +// +// The canonical Content-Type request header value for this message is +// "application/x-protobuf;proto=io.prometheus.write.v2.Request" +// +// NOTE: gogoproto options might change in future for this file, they +// are not part of the spec proto (they only modify the generated Go code, not +// the serialized message). See: https://github.com/prometheus/prometheus/issues/11908 +type Request struct { + // symbols contains a de-duplicated array of string elements used for various + // items in a Request message, like labels and metadata items. For the sender's convenience + // around empty values for optional fields like unit_ref, symbols array MUST start with + // empty string. + // + // To decode each of the symbolized strings, referenced, by "ref(s)" suffix, you + // need to lookup the actual string by index from symbols array. The order of + // strings is up to the sender. The receiver should not assume any particular encoding. + Symbols []string `protobuf:"bytes,4,rep,name=symbols,proto3" json:"symbols,omitempty"` + // timeseries represents an array of distinct series with 0 or more samples. + Timeseries []TimeSeries `protobuf:"bytes,5,rep,name=timeseries,proto3" json:"timeseries"` +} + +func (m *Request) Reset() { *m = Request{} } +func (m *Request) String() string { return proto.CompactTextString(m) } +func (*Request) ProtoMessage() {} +func (*Request) Descriptor() ([]byte, []int) { + return fileDescriptor_9c40b9525775fffe, []int{0} +} +func (m *Request) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Request) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Request.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 *Request) XXX_Merge(src proto.Message) { + xxx_messageInfo_Request.Merge(m, src) +} +func (m *Request) XXX_Size() int { + return m.Size() +} +func (m *Request) XXX_DiscardUnknown() { + xxx_messageInfo_Request.DiscardUnknown(m) +} + +var xxx_messageInfo_Request proto.InternalMessageInfo + +func (m *Request) GetSymbols() []string { + if m != nil { + return m.Symbols + } + return nil +} + +func (m *Request) GetTimeseries() []TimeSeries { + if m != nil { + return m.Timeseries + } + return nil +} + +// TimeSeries represents a single series. +type TimeSeries struct { + // labels_refs is a list of label name-value pair references, encoded + // as indices to the Request.symbols array. This list's length is always + // a multiple of two, and the underlying labels should be sorted lexicographically. + // + // Note that there might be multiple TimeSeries objects in the same + // Requests with the same labels e.g. for different exemplars, metadata + // or created timestamp. + LabelsRefs []uint32 `protobuf:"varint,1,rep,packed,name=labels_refs,json=labelsRefs,proto3" json:"labels_refs,omitempty"` + // Timeseries messages can either specify samples or (native) histogram samples + // (histogram field), but not both. For a typical sender (real-time metric + // streaming), in healthy cases, there will be only one sample or histogram. + // + // Samples and histograms are sorted by timestamp (older first). + Samples []Sample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples"` + Histograms []Histogram `protobuf:"bytes,3,rep,name=histograms,proto3" json:"histograms"` + // exemplars represents an optional set of exemplars attached to this series' samples. + Exemplars []Exemplar `protobuf:"bytes,4,rep,name=exemplars,proto3" json:"exemplars"` + // metadata represents the metadata associated with the given series' samples. + Metadata Metadata `protobuf:"bytes,5,opt,name=metadata,proto3" json:"metadata"` + // created_timestamp represents an optional created timestamp associated with + // this series' samples in ms format, typically for counter or histogram type + // metrics. Created timestamp represents the time when the counter started + // counting (sometimes referred to as start timestamp), which can increase + // the accuracy of query results. + // + // Note that some receivers might require this and in return fail to + // ingest such samples within the Request. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + // + // Note that the "optional" keyword is omitted due to + // https://cloud.google.com/apis/design/design_patterns.md#optional_primitive_fields + // Zero value means value not set. If you need to use exactly zero value for + // the timestamp, use 1 millisecond before or after. + CreatedTimestamp int64 `protobuf:"varint,6,opt,name=created_timestamp,json=createdTimestamp,proto3" json:"created_timestamp,omitempty"` +} + +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_9c40b9525775fffe, []int{1} +} +func (m *TimeSeries) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TimeSeries) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TimeSeries.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 *TimeSeries) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSeries.Merge(m, src) +} +func (m *TimeSeries) XXX_Size() int { + return m.Size() +} +func (m *TimeSeries) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSeries.DiscardUnknown(m) +} + +var xxx_messageInfo_TimeSeries proto.InternalMessageInfo + +func (m *TimeSeries) GetLabelsRefs() []uint32 { + if m != nil { + return m.LabelsRefs + } + return nil +} + +func (m *TimeSeries) GetSamples() []Sample { + if m != nil { + return m.Samples + } + return nil +} + +func (m *TimeSeries) GetHistograms() []Histogram { + if m != nil { + return m.Histograms + } + return nil +} + +func (m *TimeSeries) GetExemplars() []Exemplar { + if m != nil { + return m.Exemplars + } + return nil +} + +func (m *TimeSeries) GetMetadata() Metadata { + if m != nil { + return m.Metadata + } + return Metadata{} +} + +func (m *TimeSeries) GetCreatedTimestamp() int64 { + if m != nil { + return m.CreatedTimestamp + } + return 0 +} + +// Exemplar is an additional information attached to some series' samples. +// It is typically used to attach an example trace or request ID associated with +// the metric changes. +type Exemplar struct { + // labels_refs is an optional list of label name-value pair references, encoded + // as indices to the Request.symbols array. This list's len is always + // a multiple of 2, and the underlying labels should be sorted lexicographically. + // If the exemplar references a trace it should use the `trace_id` label name, as a best practice. + LabelsRefs []uint32 `protobuf:"varint,1,rep,packed,name=labels_refs,json=labelsRefs,proto3" json:"labels_refs,omitempty"` + // value represents an exact example value. This can be useful when the exemplar + // is attached to a histogram, which only gives an estimated value through buckets. + Value float64 `protobuf:"fixed64,2,opt,name=value,proto3" json:"value,omitempty"` + // timestamp represents the timestamp of the exemplar in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + Timestamp int64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` +} + +func (m *Exemplar) Reset() { *m = Exemplar{} } +func (m *Exemplar) String() string { return proto.CompactTextString(m) } +func (*Exemplar) ProtoMessage() {} +func (*Exemplar) Descriptor() ([]byte, []int) { + return fileDescriptor_9c40b9525775fffe, []int{2} +} +func (m *Exemplar) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Exemplar) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Exemplar.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 *Exemplar) XXX_Merge(src proto.Message) { + xxx_messageInfo_Exemplar.Merge(m, src) +} +func (m *Exemplar) XXX_Size() int { + return m.Size() +} +func (m *Exemplar) XXX_DiscardUnknown() { + xxx_messageInfo_Exemplar.DiscardUnknown(m) +} + +var xxx_messageInfo_Exemplar proto.InternalMessageInfo + +func (m *Exemplar) GetLabelsRefs() []uint32 { + if m != nil { + return m.LabelsRefs + } + return nil +} + +func (m *Exemplar) GetValue() float64 { + if m != nil { + return m.Value + } + return 0 +} + +func (m *Exemplar) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +// Sample represents series sample. +type Sample struct { + // value of the sample. + Value float64 `protobuf:"fixed64,1,opt,name=value,proto3" json:"value,omitempty"` + // timestamp represents timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` +} + +func (m *Sample) Reset() { *m = Sample{} } +func (m *Sample) String() string { return proto.CompactTextString(m) } +func (*Sample) ProtoMessage() {} +func (*Sample) Descriptor() ([]byte, []int) { + return fileDescriptor_9c40b9525775fffe, []int{3} +} +func (m *Sample) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Sample) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Sample.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 *Sample) XXX_Merge(src proto.Message) { + xxx_messageInfo_Sample.Merge(m, src) +} +func (m *Sample) XXX_Size() int { + return m.Size() +} +func (m *Sample) XXX_DiscardUnknown() { + xxx_messageInfo_Sample.DiscardUnknown(m) +} + +var xxx_messageInfo_Sample proto.InternalMessageInfo + +func (m *Sample) GetValue() float64 { + if m != nil { + return m.Value + } + return 0 +} + +func (m *Sample) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +// Metadata represents the metadata associated with the given series' samples. +type Metadata struct { + Type Metadata_MetricType `protobuf:"varint,1,opt,name=type,proto3,enum=prometheus_writev2.Metadata_MetricType" json:"type,omitempty"` + // help_ref is a reference to the Request.symbols array representing help + // text for the metric. Help is optional, reference should point to an empty string in + // such a case. + HelpRef uint32 `protobuf:"varint,3,opt,name=help_ref,json=helpRef,proto3" json:"help_ref,omitempty"` + // unit_ref is a reference to the Request.symbols array representing a unit + // for the metric. Unit is optional, reference should point to an empty string in + // such a case. + UnitRef uint32 `protobuf:"varint,4,opt,name=unit_ref,json=unitRef,proto3" json:"unit_ref,omitempty"` +} + +func (m *Metadata) Reset() { *m = Metadata{} } +func (m *Metadata) String() string { return proto.CompactTextString(m) } +func (*Metadata) ProtoMessage() {} +func (*Metadata) Descriptor() ([]byte, []int) { + return fileDescriptor_9c40b9525775fffe, []int{4} +} +func (m *Metadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Metadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Metadata.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 *Metadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_Metadata.Merge(m, src) +} +func (m *Metadata) XXX_Size() int { + return m.Size() +} +func (m *Metadata) XXX_DiscardUnknown() { + xxx_messageInfo_Metadata.DiscardUnknown(m) +} + +var xxx_messageInfo_Metadata proto.InternalMessageInfo + +func (m *Metadata) GetType() Metadata_MetricType { + if m != nil { + return m.Type + } + return Metadata_METRIC_TYPE_UNSPECIFIED +} + +func (m *Metadata) GetHelpRef() uint32 { + if m != nil { + return m.HelpRef + } + return 0 +} + +func (m *Metadata) GetUnitRef() uint32 { + if m != nil { + return m.UnitRef + } + 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 -53 and numbers in range of -4 <= n <= 8. More valid numbers might be + // added in future for new bucketing layouts. + // + // The schema equal to -53 means custom buckets. See + // custom_values field description for more details. + // + // Values between -4 and 8 represent base-2 bucket schema, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n (n is schema value) logarithmic buckets. Or in other words, + // each bucket boundary is the previous boundary times 2^(2^-n). + 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"` + // 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. + // + // In case of custom buckets (-53 schema value) the positive buckets are interpreted as follows: + // * The span offset+length points to an the index of the custom_values array + // or +Inf if pointing to the len of the array. + // * The counts and deltas have the same meaning as for exponential histograms. + PositiveSpans []BucketSpan `protobuf:"bytes,11,rep,name=positive_spans,json=positiveSpans,proto3" json:"positive_spans"` + // 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_writev2.Histogram_ResetHint" json:"reset_hint,omitempty"` + // timestamp represents timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + Timestamp int64 `protobuf:"varint,15,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + // custom_values is an additional field used by non-exponential bucketing layouts. + // + // For custom buckets (-53 schema value) custom_values specify monotonically + // increasing upper inclusive boundaries for the bucket counts with arbitrary + // widths for this histogram. In other words, custom_values represents custom, + // explicit bucketing that could have been converted from the classic histograms. + // + // Those bounds are then referenced by spans in positive_spans with corresponding positive + // counts of deltas (refer to positive_spans for more details). This way we can + // have encode sparse histograms with custom bucketing (many buckets are often + // not used). + // + // Note that for custom bounds, even negative observations are placed in the positive + // counts to simplify the implementation and avoid ambiguity of where to place + // an underflow bucket, e.g. (-2, 1]. Therefore negative buckets and + // the zero bucket are unused, if the schema indicates custom bucketing. + // + // For each upper boundary the previous boundary represent the lower exclusive + // boundary for that bucket. The first element is the upper inclusive boundary + // for the first bucket, which implicitly has a lower inclusive bound of -Inf. + // This is similar to "le" label semantics on classic histograms. You may add a + // bucket with an upper bound of 0 to make sure that you really have no negative + // observations, but in practice, native histogram rendering will show both with + // or without first upper boundary 0 and no negative counts as the same case. + // + // The last element is not only the upper inclusive bound of the last regular + // bucket, but implicitly the lower exclusive bound of the +Inf bucket. + CustomValues []float64 `protobuf:"fixed64,16,rep,packed,name=custom_values,json=customValues,proto3" json:"custom_values,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_9c40b9525775fffe, []int{5} +} +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_RESET_HINT_UNSPECIFIED +} + +func (m *Histogram) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *Histogram) GetCustomValues() []float64 { + if m != nil { + return m.CustomValues + } + return nil +} + +// 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_9c40b9525775fffe, []int{6} +} +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 +} + +func init() { + proto.RegisterEnum("prometheus_writev2.Metadata_MetricType", Metadata_MetricType_name, Metadata_MetricType_value) + proto.RegisterEnum("prometheus_writev2.Histogram_ResetHint", Histogram_ResetHint_name, Histogram_ResetHint_value) + proto.RegisterType((*Request)(nil), "prometheus_writev2.Request") + proto.RegisterType((*TimeSeries)(nil), "prometheus_writev2.TimeSeries") + proto.RegisterType((*Exemplar)(nil), "prometheus_writev2.Exemplar") + proto.RegisterType((*Sample)(nil), "prometheus_writev2.Sample") + proto.RegisterType((*Metadata)(nil), "prometheus_writev2.Metadata") + proto.RegisterType((*Histogram)(nil), "prometheus_writev2.Histogram") + proto.RegisterType((*BucketSpan)(nil), "prometheus_writev2.BucketSpan") +} + +func init() { + proto.RegisterFile("store/storepb/writev2pb/types.proto", fileDescriptor_9c40b9525775fffe) +} + +var fileDescriptor_9c40b9525775fffe = []byte{ + // 940 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xdd, 0x6e, 0xe3, 0x44, + 0x14, 0xce, 0xc4, 0xf9, 0x3d, 0x69, 0xb2, 0xce, 0xd0, 0xdd, 0xf5, 0x96, 0xdd, 0x6c, 0x48, 0x05, + 0x1b, 0x81, 0xd4, 0x4a, 0xe5, 0x0e, 0x56, 0x88, 0x26, 0x75, 0x9b, 0x80, 0x92, 0xae, 0x26, 0x2e, + 0x52, 0xb9, 0xb1, 0x9c, 0x64, 0x92, 0x58, 0xd8, 0xb1, 0xf1, 0x4c, 0x0a, 0xe5, 0x29, 0x78, 0x1c, + 0x1e, 0x61, 0x2f, 0xf7, 0x92, 0x9b, 0x22, 0xd4, 0xbe, 0x08, 0x9a, 0xf1, 0x6f, 0xff, 0x40, 0xdc, + 0x44, 0x33, 0xdf, 0xf9, 0xbe, 0x73, 0x3e, 0x9f, 0x9c, 0x63, 0xc3, 0x2e, 0xe3, 0x5e, 0x40, 0xf7, + 0xe5, 0xaf, 0x3f, 0xdd, 0xff, 0x25, 0xb0, 0x39, 0xbd, 0x38, 0xf0, 0xa7, 0xfb, 0xfc, 0xd2, 0xa7, + 0x6c, 0xcf, 0x0f, 0x3c, 0xee, 0x61, 0xec, 0x07, 0x9e, 0x4b, 0xf9, 0x8a, 0x6e, 0x98, 0x19, 0x31, + 0x76, 0xb6, 0x97, 0xde, 0xd2, 0x93, 0xe1, 0x7d, 0x71, 0x0a, 0x99, 0x1d, 0x0f, 0xca, 0x84, 0xfe, + 0xbc, 0xa1, 0x8c, 0x63, 0x0d, 0xca, 0xec, 0xd2, 0x9d, 0x7a, 0x0e, 0xd3, 0x0a, 0x6d, 0xa5, 0x5b, + 0x25, 0xf1, 0x15, 0x1f, 0x01, 0x70, 0xdb, 0xa5, 0x8c, 0x06, 0x36, 0x65, 0x5a, 0xb1, 0xad, 0x74, + 0x6b, 0x07, 0xad, 0xbd, 0xfb, 0x35, 0xf6, 0x0c, 0xdb, 0xa5, 0x13, 0xc9, 0xea, 0x15, 0xde, 0xff, + 0xf5, 0x3a, 0x47, 0x32, 0xba, 0xef, 0x0a, 0x15, 0xa4, 0x16, 0x3a, 0x57, 0x79, 0x80, 0x94, 0x86, + 0x5f, 0x43, 0xcd, 0xb1, 0xa6, 0xd4, 0x61, 0x66, 0x40, 0x17, 0x4c, 0x43, 0x6d, 0xa5, 0x5b, 0x27, + 0x10, 0x42, 0x84, 0x2e, 0x18, 0xfe, 0x0a, 0xca, 0xcc, 0x72, 0x7d, 0x87, 0x32, 0x2d, 0x2f, 0x0b, + 0xef, 0x3c, 0x54, 0x78, 0x22, 0x29, 0x51, 0xd1, 0x58, 0x80, 0xfb, 0x00, 0x2b, 0x9b, 0x71, 0x6f, + 0x19, 0x58, 0x2e, 0xd3, 0x14, 0x29, 0x7f, 0xf5, 0x90, 0x7c, 0x10, 0xb3, 0x62, 0xdb, 0xa9, 0x0c, + 0x7f, 0x0b, 0x55, 0xfa, 0x2b, 0x75, 0x7d, 0xc7, 0x0a, 0xc2, 0xc6, 0xd4, 0x0e, 0x5e, 0x3e, 0x94, + 0x43, 0x8f, 0x48, 0x51, 0x8a, 0x54, 0x84, 0xbf, 0x81, 0x8a, 0x4b, 0xb9, 0x35, 0xb7, 0xb8, 0xa5, + 0x15, 0xdb, 0xe8, 0xb1, 0x04, 0xa3, 0x88, 0x13, 0x25, 0x48, 0x34, 0xf8, 0x0b, 0x68, 0xce, 0x02, + 0x6a, 0x71, 0x3a, 0x37, 0x65, 0x3b, 0xb9, 0xe5, 0xfa, 0x5a, 0xa9, 0x8d, 0xba, 0x0a, 0x51, 0xa3, + 0x80, 0x11, 0xe3, 0x1d, 0x13, 0x2a, 0xb1, 0x93, 0xff, 0x6e, 0xee, 0x36, 0x14, 0x2f, 0x2c, 0x67, + 0x43, 0xb5, 0x7c, 0x1b, 0x75, 0x11, 0x09, 0x2f, 0xf8, 0x25, 0x54, 0xd3, 0x3a, 0x8a, 0xac, 0x93, + 0x02, 0x9d, 0xb7, 0x50, 0x0a, 0xbb, 0x9d, 0xaa, 0xd1, 0xa3, 0xea, 0xfc, 0x5d, 0xf5, 0x55, 0x1e, + 0x2a, 0xf1, 0x83, 0xe2, 0xaf, 0xa1, 0x20, 0xa6, 0x56, 0xea, 0x1b, 0x07, 0x6f, 0xfe, 0xad, 0x29, + 0xe2, 0x10, 0xd8, 0x33, 0xe3, 0xd2, 0xa7, 0x44, 0x8a, 0xf0, 0x0b, 0xa8, 0xac, 0xa8, 0xe3, 0x8b, + 0x47, 0x93, 0x26, 0xeb, 0xa4, 0x2c, 0xee, 0x84, 0x2e, 0x44, 0x68, 0xb3, 0xb6, 0xb9, 0x0c, 0x15, + 0xc2, 0x90, 0xb8, 0x13, 0xba, 0xe8, 0x5c, 0x21, 0x80, 0x34, 0x15, 0xfe, 0x18, 0x9e, 0x8f, 0x74, + 0x83, 0x0c, 0xfb, 0xa6, 0x71, 0xfe, 0x4e, 0x37, 0xcf, 0xc6, 0x93, 0x77, 0x7a, 0x7f, 0x78, 0x3c, + 0xd4, 0x8f, 0xd4, 0x1c, 0x7e, 0x0e, 0x1f, 0x65, 0x83, 0xfd, 0xd3, 0xb3, 0xb1, 0xa1, 0x13, 0x15, + 0xe1, 0xa7, 0xd0, 0xcc, 0x06, 0x4e, 0x0e, 0xcf, 0x4e, 0x74, 0x35, 0x8f, 0x5f, 0xc0, 0xd3, 0x2c, + 0x3c, 0x18, 0x4e, 0x8c, 0xd3, 0x13, 0x72, 0x38, 0x52, 0x15, 0xdc, 0x82, 0x9d, 0x7b, 0x8a, 0x34, + 0x5e, 0xb8, 0x5b, 0x6a, 0x72, 0x36, 0x1a, 0x1d, 0x92, 0x73, 0xb5, 0x88, 0xb7, 0x41, 0xcd, 0x06, + 0x86, 0xe3, 0xe3, 0x53, 0xb5, 0x84, 0x35, 0xd8, 0xbe, 0x45, 0x37, 0x0e, 0x0d, 0x7d, 0xa2, 0x1b, + 0x6a, 0xb9, 0xf3, 0x47, 0x09, 0xaa, 0xc9, 0x34, 0xe3, 0x57, 0x50, 0x9d, 0x79, 0x9b, 0x35, 0x37, + 0xed, 0x35, 0x97, 0x5d, 0x2e, 0x0c, 0x72, 0xa4, 0x22, 0xa1, 0xe1, 0x9a, 0xe3, 0x4f, 0xa0, 0x16, + 0x86, 0x17, 0x8e, 0x67, 0xf1, 0x70, 0x08, 0x06, 0x39, 0x02, 0x12, 0x3c, 0x16, 0x18, 0x56, 0x41, + 0x61, 0x1b, 0x57, 0x36, 0x18, 0x11, 0x71, 0xc4, 0xcf, 0xa0, 0xc4, 0x66, 0x2b, 0xea, 0x5a, 0xb2, + 0xb5, 0x4d, 0x12, 0xdd, 0xf0, 0xa7, 0xd0, 0xf8, 0x8d, 0x06, 0x9e, 0xc9, 0x57, 0x01, 0x65, 0x2b, + 0xcf, 0x99, 0xcb, 0x59, 0x47, 0xa4, 0x2e, 0x50, 0x23, 0x06, 0xf1, 0x67, 0x11, 0x2d, 0xf5, 0x55, + 0x92, 0xbe, 0x10, 0xd9, 0x12, 0x78, 0x3f, 0xf6, 0xf6, 0x39, 0xa8, 0x19, 0x5e, 0x68, 0xb0, 0x2c, + 0x0d, 0x22, 0xd2, 0x48, 0x98, 0xa1, 0xc9, 0xef, 0xa1, 0xb1, 0xa6, 0x4b, 0x8b, 0xdb, 0x17, 0xd4, + 0x64, 0xbe, 0xb5, 0x66, 0x5a, 0xe5, 0xf1, 0x77, 0x54, 0x6f, 0x33, 0xfb, 0x89, 0xf2, 0x89, 0x6f, + 0xad, 0xa3, 0x45, 0xab, 0xc7, 0x5a, 0x81, 0x31, 0xfc, 0x06, 0x9e, 0x24, 0xc9, 0xe6, 0xd4, 0xe1, + 0x16, 0xd3, 0xaa, 0x6d, 0xa5, 0x8b, 0x49, 0x52, 0xe3, 0x48, 0xa2, 0xb7, 0x88, 0xd2, 0x25, 0xd3, + 0xa0, 0xad, 0x74, 0x51, 0x4a, 0x94, 0x16, 0x99, 0xb0, 0xe7, 0x7b, 0xcc, 0xce, 0xd8, 0xab, 0xfd, + 0x1f, 0x7b, 0xb1, 0x36, 0xb1, 0x97, 0x24, 0x8b, 0xec, 0x6d, 0x85, 0xf6, 0x62, 0x38, 0xb5, 0x97, + 0x10, 0x23, 0x7b, 0xf5, 0xd0, 0x5e, 0x0c, 0x47, 0xf6, 0x8e, 0x01, 0x02, 0xca, 0x28, 0x37, 0x57, + 0xe2, 0xdf, 0x68, 0x3c, 0xbe, 0x8b, 0xc9, 0x5c, 0xed, 0x11, 0xc1, 0x1f, 0xd8, 0x6b, 0x4e, 0xaa, + 0x41, 0x7c, 0xbc, 0xbd, 0xf8, 0x4f, 0xee, 0x2c, 0x3e, 0xde, 0x85, 0xfa, 0x6c, 0xc3, 0xb8, 0xe7, + 0x9a, 0xf2, 0x35, 0xc1, 0x34, 0x55, 0x9a, 0xd9, 0x0a, 0xc1, 0x1f, 0x24, 0xd6, 0x99, 0x43, 0x35, + 0x49, 0x8d, 0x77, 0xe0, 0x19, 0x11, 0x53, 0x6d, 0x0e, 0x86, 0x63, 0xe3, 0xce, 0x6a, 0x62, 0x68, + 0x64, 0x62, 0xe7, 0xfa, 0x44, 0x45, 0xb8, 0x09, 0xf5, 0x0c, 0x36, 0x3e, 0x55, 0xf3, 0x62, 0x7b, + 0x32, 0x50, 0xb8, 0xa7, 0x4a, 0xaf, 0x0c, 0x45, 0xd9, 0x90, 0xde, 0x16, 0x40, 0x3a, 0x63, 0x9d, + 0xb7, 0x00, 0x69, 0xf3, 0xc5, 0x98, 0x7b, 0x8b, 0x05, 0xa3, 0xe1, 0xde, 0x34, 0x49, 0x74, 0x13, + 0xb8, 0x43, 0xd7, 0x4b, 0xbe, 0x92, 0xeb, 0x52, 0x27, 0xd1, 0xad, 0xb7, 0xfb, 0xfe, 0xba, 0x85, + 0x3e, 0x5c, 0xb7, 0xd0, 0xdf, 0xd7, 0x2d, 0xf4, 0xfb, 0x4d, 0x2b, 0xf7, 0xe1, 0xa6, 0x95, 0xfb, + 0xf3, 0xa6, 0x95, 0xfb, 0xb1, 0x9a, 0x7c, 0xa3, 0xa7, 0x25, 0xf9, 0xd1, 0xfd, 0xf2, 0x9f, 0x00, + 0x00, 0x00, 0xff, 0xff, 0xf4, 0xcc, 0x05, 0x0b, 0xc5, 0x07, 0x00, 0x00, +} + +func (m *Request) 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 *Request) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Request) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Timeseries) > 0 { + for iNdEx := len(m.Timeseries) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Timeseries[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + } + if len(m.Symbols) > 0 { + for iNdEx := len(m.Symbols) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Symbols[iNdEx]) + copy(dAtA[i:], m.Symbols[iNdEx]) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Symbols[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + 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 m.CreatedTimestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.CreatedTimestamp)) + i-- + dAtA[i] = 0x30 + } + { + size, err := m.Metadata.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + 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] = 0x22 + } + } + 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] = 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.LabelsRefs) > 0 { + dAtA3 := make([]byte, len(m.LabelsRefs)*10) + var j2 int + for _, num := range m.LabelsRefs { + for num >= 1<<7 { + dAtA3[j2] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j2++ + } + dAtA3[j2] = uint8(num) + j2++ + } + i -= j2 + copy(dAtA[i:], dAtA3[:j2]) + i = encodeVarintTypes(dAtA, i, uint64(j2)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Exemplar) 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 *Exemplar) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Exemplar) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Timestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x18 + } + if m.Value != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) + i-- + dAtA[i] = 0x11 + } + if len(m.LabelsRefs) > 0 { + dAtA5 := make([]byte, len(m.LabelsRefs)*10) + var j4 int + for _, num := range m.LabelsRefs { + for num >= 1<<7 { + dAtA5[j4] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j4++ + } + dAtA5[j4] = uint8(num) + j4++ + } + i -= j4 + copy(dAtA[i:], dAtA5[:j4]) + i = encodeVarintTypes(dAtA, i, uint64(j4)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Sample) 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 *Sample) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Timestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x10 + } + if m.Value != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) + i-- + dAtA[i] = 0x9 + } + return len(dAtA) - i, nil +} + +func (m *Metadata) 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 *Metadata) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Metadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.UnitRef != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.UnitRef)) + i-- + dAtA[i] = 0x20 + } + if m.HelpRef != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.HelpRef)) + i-- + dAtA[i] = 0x18 + } + if m.Type != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Histogram) 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 *Histogram) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.CustomValues) > 0 { + for iNdEx := len(m.CustomValues) - 1; iNdEx >= 0; iNdEx-- { + f6 := math.Float64bits(float64(m.CustomValues[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f6)) + } + i = encodeVarintTypes(dAtA, i, uint64(len(m.CustomValues)*8)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + 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-- { + f7 := math.Float64bits(float64(m.PositiveCounts[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f7)) + } + i = encodeVarintTypes(dAtA, i, uint64(len(m.PositiveCounts)*8)) + i-- + dAtA[i] = 0x6a + } + if len(m.PositiveDeltas) > 0 { + var j8 int + dAtA10 := make([]byte, len(m.PositiveDeltas)*10) + for _, num := range m.PositiveDeltas { + x9 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x9 >= 1<<7 { + dAtA10[j8] = uint8(uint64(x9)&0x7f | 0x80) + j8++ + x9 >>= 7 + } + dAtA10[j8] = uint8(x9) + j8++ + } + i -= j8 + copy(dAtA[i:], dAtA10[:j8]) + i = encodeVarintTypes(dAtA, i, uint64(j8)) + i-- + dAtA[i] = 0x62 + } + if len(m.PositiveSpans) > 0 { + for iNdEx := len(m.PositiveSpans) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.PositiveSpans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x5a + } + } + if len(m.NegativeCounts) > 0 { + for iNdEx := len(m.NegativeCounts) - 1; iNdEx >= 0; iNdEx-- { + f11 := math.Float64bits(float64(m.NegativeCounts[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f11)) + } + i = encodeVarintTypes(dAtA, i, uint64(len(m.NegativeCounts)*8)) + i-- + dAtA[i] = 0x52 + } + if len(m.NegativeDeltas) > 0 { + var j12 int + dAtA14 := make([]byte, len(m.NegativeDeltas)*10) + for _, num := range m.NegativeDeltas { + x13 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x13 >= 1<<7 { + dAtA14[j12] = uint8(uint64(x13)&0x7f | 0x80) + j12++ + x13 >>= 7 + } + dAtA14[j12] = uint8(x13) + j12++ + } + i -= j12 + copy(dAtA[i:], dAtA14[:j12]) + i = encodeVarintTypes(dAtA, i, uint64(j12)) + i-- + dAtA[i] = 0x4a + } + if len(m.NegativeSpans) > 0 { + for iNdEx := len(m.NegativeSpans) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.NegativeSpans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + } + 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 + } + } + } + return len(dAtA) - i, nil +} + +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 encodeVarintTypes(dAtA []byte, offset int, v uint64) int { + offset -= sovTypes(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Request) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Symbols) > 0 { + for _, s := range m.Symbols { + l = len(s) + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.Timeseries) > 0 { + for _, e := range m.Timeseries { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + return n +} + +func (m *TimeSeries) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.LabelsRefs) > 0 { + l = 0 + for _, e := range m.LabelsRefs { + l += sovTypes(uint64(e)) + } + n += 1 + sovTypes(uint64(l)) + l + } + if len(m.Samples) > 0 { + for _, e := range m.Samples { + l = e.Size() + 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)) + } + } + if len(m.Exemplars) > 0 { + for _, e := range m.Exemplars { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + l = m.Metadata.Size() + n += 1 + l + sovTypes(uint64(l)) + if m.CreatedTimestamp != 0 { + n += 1 + sovTypes(uint64(m.CreatedTimestamp)) + } + return n +} + +func (m *Exemplar) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.LabelsRefs) > 0 { + l = 0 + for _, e := range m.LabelsRefs { + l += sovTypes(uint64(e)) + } + n += 1 + sovTypes(uint64(l)) + l + } + if m.Value != 0 { + n += 9 + } + if m.Timestamp != 0 { + n += 1 + sovTypes(uint64(m.Timestamp)) + } + return n +} + +func (m *Sample) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Value != 0 { + n += 9 + } + if m.Timestamp != 0 { + n += 1 + sovTypes(uint64(m.Timestamp)) + } + return n +} + +func (m *Metadata) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Type != 0 { + n += 1 + sovTypes(uint64(m.Type)) + } + if m.HelpRef != 0 { + n += 1 + sovTypes(uint64(m.HelpRef)) + } + if m.UnitRef != 0 { + n += 1 + sovTypes(uint64(m.UnitRef)) + } + 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)) + } + if len(m.CustomValues) > 0 { + n += 2 + sovTypes(uint64(len(m.CustomValues)*8)) + len(m.CustomValues)*8 + } + 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 sovTypes(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozTypes(x uint64) (n int) { + return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Request) 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: Request: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Request: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Symbols", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Symbols = append(m.Symbols, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timeseries", 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.Timeseries = append(m.Timeseries, TimeSeries{}) + if err := m.Timeseries[len(m.Timeseries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LabelsRefs = append(m.LabelsRefs, 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.LabelsRefs) == 0 { + m.LabelsRefs = make([]uint32, 0, elementCount) + } + for iNdEx < postIndex { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LabelsRefs = append(m.LabelsRefs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field LabelsRefs", wireType) + } + 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 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 + case 4: + 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 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", 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 + } + if err := m.Metadata.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CreatedTimestamp", wireType) + } + m.CreatedTimestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CreatedTimestamp |= 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 *Exemplar) 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: Exemplar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Exemplar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LabelsRefs = append(m.LabelsRefs, 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.LabelsRefs) == 0 { + m.LabelsRefs = make([]uint32, 0, elementCount) + } + for iNdEx < postIndex { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LabelsRefs = append(m.LabelsRefs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field LabelsRefs", wireType) + } + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Value = float64(math.Float64frombits(v)) + case 3: + 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 *Sample) 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: Sample: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Sample: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Value = float64(math.Float64frombits(v)) + case 2: + 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 *Metadata) 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: Metadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Metadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= Metadata_MetricType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field HelpRef", wireType) + } + m.HelpRef = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.HelpRef |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UnitRef", wireType) + } + m.UnitRef = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.UnitRef |= 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 *Histogram) 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: Histogram: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Histogram: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CountInt", wireType) + } + 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 + } + } + m.Count = &Histogram_CountInt{v} + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field CountFloat", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + 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) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + 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 + } + 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.Schema = v + case 5: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroThreshold", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + 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 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 + } + } + m.ZeroCount = &Histogram_ZeroCountInt{v} + case 7: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroCountFloat", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + 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 + } + } + case 16: + 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.CustomValues = append(m.CustomValues, 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.CustomValues) == 0 { + m.CustomValues = 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.CustomValues = append(m.CustomValues, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field CustomValues", wireType) + } + 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 skipTypes(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthTypes + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupTypes + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthTypes + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupTypes = fmt.Errorf("proto: unexpected end of group") +) diff --git a/pkg/store/storepb/writev2pb/types.proto b/pkg/store/storepb/writev2pb/types.proto new file mode 100644 index 0000000000..0f8cb50773 --- /dev/null +++ b/pkg/store/storepb/writev2pb/types.proto @@ -0,0 +1,265 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +// Copyright 2024 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// NOTE: This file is also available on https://buf.build/prometheus/prometheus/docs/main:io.prometheus.write.v2 + +syntax = "proto3"; +package prometheus_writev2; + +option go_package = "writev2pb"; + +import "gogoproto/gogo.proto"; + +// Do not generate XXX fields to reduce memory footprint and opening a door +// for zero-copy casts to/from prometheus data types. +option (gogoproto.goproto_unkeyed_all) = false; +option (gogoproto.goproto_unrecognized_all) = false; +option (gogoproto.goproto_sizecache_all) = false; + + +// Request represents a request to write the given timeseries to a remote destination. +// This message was introduced in the Remote Write 2.0 specification: +// https://prometheus.io/docs/concepts/remote_write_spec_2_0/ +// +// The canonical Content-Type request header value for this message is +// "application/x-protobuf;proto=io.prometheus.write.v2.Request" +// +// NOTE: gogoproto options might change in future for this file, they +// are not part of the spec proto (they only modify the generated Go code, not +// the serialized message). See: https://github.com/prometheus/prometheus/issues/11908 +message Request { + // Since Request supersedes 1.0 spec's prometheus.WriteRequest, we reserve the top-down message + // for the deterministic interop between those two, see types_test.go for details. + // Generally it's not needed, because Receivers must use the Content-Type header, but we want to + // be sympathetic to adopters with mistaken implementations and have deterministic error (empty + // message if you use the wrong proto schema). + reserved 1 to 3; + + // symbols contains a de-duplicated array of string elements used for various + // items in a Request message, like labels and metadata items. For the sender's convenience + // around empty values for optional fields like unit_ref, symbols array MUST start with + // empty string. + // + // To decode each of the symbolized strings, referenced, by "ref(s)" suffix, you + // need to lookup the actual string by index from symbols array. The order of + // strings is up to the sender. The receiver should not assume any particular encoding. + repeated string symbols = 4; + // timeseries represents an array of distinct series with 0 or more samples. + repeated TimeSeries timeseries = 5 [(gogoproto.nullable) = false]; +} + +// TimeSeries represents a single series. +message TimeSeries { + // labels_refs is a list of label name-value pair references, encoded + // as indices to the Request.symbols array. This list's length is always + // a multiple of two, and the underlying labels should be sorted lexicographically. + // + // Note that there might be multiple TimeSeries objects in the same + // Requests with the same labels e.g. for different exemplars, metadata + // or created timestamp. + repeated uint32 labels_refs = 1; + + // Timeseries messages can either specify samples or (native) histogram samples + // (histogram field), but not both. For a typical sender (real-time metric + // streaming), in healthy cases, there will be only one sample or histogram. + // + // Samples and histograms are sorted by timestamp (older first). + repeated Sample samples = 2 [(gogoproto.nullable) = false]; + repeated Histogram histograms = 3 [(gogoproto.nullable) = false]; + + // exemplars represents an optional set of exemplars attached to this series' samples. + repeated Exemplar exemplars = 4 [(gogoproto.nullable) = false]; + + // metadata represents the metadata associated with the given series' samples. + Metadata metadata = 5 [(gogoproto.nullable) = false]; + + // created_timestamp represents an optional created timestamp associated with + // this series' samples in ms format, typically for counter or histogram type + // metrics. Created timestamp represents the time when the counter started + // counting (sometimes referred to as start timestamp), which can increase + // the accuracy of query results. + // + // Note that some receivers might require this and in return fail to + // ingest such samples within the Request. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + // + // Note that the "optional" keyword is omitted due to + // https://cloud.google.com/apis/design/design_patterns.md#optional_primitive_fields + // Zero value means value not set. If you need to use exactly zero value for + // the timestamp, use 1 millisecond before or after. + int64 created_timestamp = 6; +} + +// Exemplar is an additional information attached to some series' samples. +// It is typically used to attach an example trace or request ID associated with +// the metric changes. +message Exemplar { + // labels_refs is an optional list of label name-value pair references, encoded + // as indices to the Request.symbols array. This list's len is always + // a multiple of 2, and the underlying labels should be sorted lexicographically. + // If the exemplar references a trace it should use the `trace_id` label name, as a best practice. + repeated uint32 labels_refs = 1; + // value represents an exact example value. This can be useful when the exemplar + // is attached to a histogram, which only gives an estimated value through buckets. + double value = 2; + // timestamp represents the timestamp of the exemplar in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + int64 timestamp = 3; +} + +// Sample represents series sample. +message Sample { + // value of the sample. + double value = 1; + // timestamp represents timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + int64 timestamp = 2; +} + +// Metadata represents the metadata associated with the given series' samples. +message Metadata { + enum MetricType { + METRIC_TYPE_UNSPECIFIED = 0; + METRIC_TYPE_COUNTER = 1; + METRIC_TYPE_GAUGE = 2; + METRIC_TYPE_HISTOGRAM = 3; + METRIC_TYPE_GAUGEHISTOGRAM = 4; + METRIC_TYPE_SUMMARY = 5; + METRIC_TYPE_INFO = 6; + METRIC_TYPE_STATESET = 7; + } + MetricType type = 1; + // help_ref is a reference to the Request.symbols array representing help + // text for the metric. Help is optional, reference should point to an empty string in + // such a case. + uint32 help_ref = 3; + // unit_ref is a reference to the Request.symbols array representing a unit + // for the metric. Unit is optional, reference should point to an empty string in + // such a case. + uint32 unit_ref = 4; +} + +// 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 { + RESET_HINT_UNSPECIFIED = 0; // Need to test for a counter reset explicitly. + RESET_HINT_YES = 1; // This is the 1st histogram after a counter reset. + RESET_HINT_NO = 2; // There was no counter reset between this and the previous Histogram. + RESET_HINT_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 -53 and numbers in range of -4 <= n <= 8. More valid numbers might be + // added in future for new bucketing layouts. + // + // The schema equal to -53 means custom buckets. See + // custom_values field description for more details. + // + // Values between -4 and 8 represent base-2 bucket schema, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n (n is schema value) logarithmic buckets. Or in other words, + // each bucket boundary is the previous boundary times 2^(2^-n). + 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 [(gogoproto.nullable) = false]; + // 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. + // + // In case of custom buckets (-53 schema value) the positive buckets are interpreted as follows: + // * The span offset+length points to an the index of the custom_values array + // or +Inf if pointing to the len of the array. + // * The counts and deltas have the same meaning as for exponential histograms. + repeated BucketSpan positive_spans = 11 [(gogoproto.nullable) = false]; + // 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 represents timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + int64 timestamp = 15; + + // custom_values is an additional field used by non-exponential bucketing layouts. + // + // For custom buckets (-53 schema value) custom_values specify monotonically + // increasing upper inclusive boundaries for the bucket counts with arbitrary + // widths for this histogram. In other words, custom_values represents custom, + // explicit bucketing that could have been converted from the classic histograms. + // + // Those bounds are then referenced by spans in positive_spans with corresponding positive + // counts of deltas (refer to positive_spans for more details). This way we can + // have encode sparse histograms with custom bucketing (many buckets are often + // not used). + // + // Note that for custom bounds, even negative observations are placed in the positive + // counts to simplify the implementation and avoid ambiguity of where to place + // an underflow bucket, e.g. (-2, 1]. Therefore negative buckets and + // the zero bucket are unused, if the schema indicates custom bucketing. + // + // For each upper boundary the previous boundary represent the lower exclusive + // boundary for that bucket. The first element is the upper inclusive boundary + // for the first bucket, which implicitly has a lower inclusive bound of -Inf. + // This is similar to "le" label semantics on classic histograms. You may add a + // bucket with an upper bound of 0 to make sure that you really have no negative + // observations, but in practice, native histogram rendering will show both with + // or without first upper boundary 0 and no negative counts as the same case. + // + // The last element is not only the upper inclusive bound of the last regular + // bucket, but implicitly the lower exclusive bound of the +Inf bucket. + repeated double custom_values = 16; +} + +// 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. +} diff --git a/scripts/genproto.sh b/scripts/genproto.sh index 419ed67880..e3c71c15e3 100755 --- a/scripts/genproto.sh +++ b/scripts/genproto.sh @@ -26,7 +26,7 @@ PATH=${PATH}:/tmp/protobin GOGOPROTO_ROOT="$(GO111MODULE=on go list -modfile=.bingo/protoc-gen-gogofast.mod -f '{{ .Dir }}' -m github.com/gogo/protobuf)" GOGOPROTO_PATH="${GOGOPROTO_ROOT}:${GOGOPROTO_ROOT}/protobuf" -DIRS="store/storepb/ store/storepb/prompb/ store/labelpb rules/rulespb targets/targetspb store/hintspb queryfrontend metadata/metadatapb exemplars/exemplarspb info/infopb api/query/querypb" +DIRS="store/storepb/ store/storepb/prompb/ store/storepb/writev2pb/ store/labelpb rules/rulespb targets/targetspb store/hintspb queryfrontend metadata/metadatapb exemplars/exemplarspb info/infopb api/query/querypb" echo "generating code" pushd "pkg" for dir in ${DIRS}; do @@ -43,6 +43,7 @@ for dir in ${DIRS}; do sed -i.bak -E 's/\"store\/storepb\"/\"github.com\/thanos-io\/thanos\/pkg\/store\/storepb\"/g' *.pb.go sed -i.bak -E 's/\"store\/labelpb\"/\"github.com\/thanos-io\/thanos\/pkg\/store\/labelpb\"/g' *.pb.go sed -i.bak -E 's/\"store\/storepb\/prompb\"/\"github.com\/thanos-io\/thanos\/pkg\/store\/storepb\/prompb\"/g' *.pb.go + sed -i.bak -E 's/\"store\/storepb\/writev2pb\"/\"github.com\/thanos-io\/thanos\/pkg\/store\/storepb\/writev2pb\"/g' *.pb.go rm -f *.bak ${GOIMPORTS_BIN} -w *.pb.go popd