diff --git a/integration/distributor_test.go b/integration/distributor_test.go index 4f2d73d3eb..27c6beb89f 100644 --- a/integration/distributor_test.go +++ b/integration/distributor_test.go @@ -376,6 +376,7 @@ overrides: func TestDistributorRemoteWrite2(t *testing.T) { queryEnd := time.Now().Round(time.Second) queryStart := queryEnd.Add(-1 * time.Hour) + queryStep := 10 * time.Minute testCases := map[string]struct { inRemoteWrite []*promRW2.Request @@ -424,7 +425,7 @@ func TestDistributorRemoteWrite2(t *testing.T) { "-distributor.ha-tracker.store": "consul", "-distributor.ha-tracker.consul.hostname": consul.NetworkHTTPEndpoint(), "-distributor.ha-tracker.prefix": "prom_ha/", - "-timeseries-unmarshal-caching-optimization-enabled": strconv.FormatBool(false), + "-timeseries-unmarshal-caching-optimization-enabled": strconv.FormatBool(false), // TODO(krajorama): add cachingUnmarshalDataEnabled testcase. } flags := mergeFlags( @@ -490,10 +491,22 @@ func TestDistributorRemoteWrite2(t *testing.T) { res, err := client.PushRW2(ser) require.NoError(t, err) - require.Equal(t, http.StatusUnsupportedMediaType, res.StatusCode) + require.True(t, res.StatusCode == http.StatusOK || res.StatusCode == http.StatusAccepted, res.Status) } - // Placeholder for actual query tests. + for q, res := range tc.queries { + result, err := client.QueryRange(q, queryStart, queryEnd, queryStep) + require.NoError(t, err) + + require.Equal(t, res.String(), result.String()) + } + + for q, expResult := range tc.exemplarQueries { + result, err := client.QueryExemplars(q, queryStart, queryEnd) + require.NoError(t, err) + + require.Equal(t, expResult, result) + } }) } } diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 4feaaa08ee..e6f45c6b9f 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -1584,7 +1584,7 @@ func (d *Distributor) push(ctx context.Context, pushReq *Request) error { return err } - d.updateReceivedMetrics(req, userID) + d.updateReceivedMetrics(ctx, req, userID) if len(req.Timeseries) == 0 && len(req.Metadata) == 0 { return nil @@ -1815,17 +1815,20 @@ func tokenForMetadata(userID string, metricName string) uint32 { return mimirpb.ShardByMetricName(userID, metricName) } -func (d *Distributor) updateReceivedMetrics(req *mimirpb.WriteRequest, userID string) { - var receivedSamples, receivedExemplars, receivedMetadata int +func (d *Distributor) updateReceivedMetrics(ctx context.Context, req *mimirpb.WriteRequest, userID string) { + var receivedSamples, receivedHistograms, receivedExemplars, receivedMetadata int for _, ts := range req.Timeseries { - receivedSamples += len(ts.TimeSeries.Samples) + len(ts.TimeSeries.Histograms) + receivedSamples += len(ts.TimeSeries.Samples) + receivedHistograms += len(ts.TimeSeries.Histograms) receivedExemplars += len(ts.TimeSeries.Exemplars) } receivedMetadata = len(req.Metadata) - d.receivedSamples.WithLabelValues(userID).Add(float64(receivedSamples)) + d.receivedSamples.WithLabelValues(userID).Add(float64(receivedSamples + receivedHistograms)) d.receivedExemplars.WithLabelValues(userID).Add(float64(receivedExemplars)) d.receivedMetadata.WithLabelValues(userID).Add(float64(receivedMetadata)) + + updateWriteResponseStatsCtx(ctx, receivedSamples, receivedHistograms, receivedExemplars) } // forReplicationSets runs f, in parallel, for all ingesters in the input replicationSets. diff --git a/pkg/distributor/push.go b/pkg/distributor/push.go index d19643cd61..02e6b38fe8 100644 --- a/pkg/distributor/push.go +++ b/pkg/distributor/push.go @@ -25,6 +25,7 @@ import ( "github.com/grafana/dskit/user" "github.com/opentracing/opentracing-go" "github.com/pkg/errors" + promRemote "github.com/prometheus/prometheus/storage/remote" "github.com/grafana/mimir/pkg/mimirpb" "github.com/grafana/mimir/pkg/util" @@ -36,6 +37,13 @@ import ( // PushFunc defines the type of the push. It is similar to http.HandlerFunc. type PushFunc func(ctx context.Context, req *Request) error +// The PushFunc might store promRemote.WriteResponseStats in the context. +type pushResponseStatsContextMarker struct{} + +var ( + PushResponseStatsContextKey = &pushResponseStatsContextMarker{} +) + // parserFunc defines how to read the body the request from an HTTP request. It takes an optional RequestBuffers. type parserFunc func(ctx context.Context, r *http.Request, maxSize int, buffers *util.RequestBuffers, req *mimirpb.PreallocWriteRequest, logger log.Logger) error @@ -151,65 +159,68 @@ func handler( } } - var supplier supplierFunc isRW2, err := isRemoteWrite2(r) if err != nil { http.Error(w, err.Error(), http.StatusBadRequest) } - if isRW2 { - supplier = func() (*mimirpb.WriteRequest, func(), error) { - // Return 415 Unsupported Media Type for remote-write v2 requests for now. This is not retryable - // unless the client switches to remote-write v1. - return nil, nil, httpgrpc.Error(http.StatusUnsupportedMediaType, "remote-write v2 is not supported") - } - } else { - supplier = func() (*mimirpb.WriteRequest, func(), error) { - rb := util.NewRequestBuffers(requestBufferPool) - var req mimirpb.PreallocWriteRequest + supplier := func() (*mimirpb.WriteRequest, func(), error) { + rb := util.NewRequestBuffers(requestBufferPool) + var req mimirpb.PreallocWriteRequest - userID, err := tenant.TenantID(ctx) - if err != nil && !errors.Is(err, user.ErrNoOrgID) { // ignore user.ErrNoOrgID - return nil, nil, errors.Wrap(err, "failed to get tenant ID") - } + req.UnmarshalFromRW2 = isRW2 - // userID might be empty if none was in the ctx, in this case just use the default setting. - if limits.MaxGlobalExemplarsPerUser(userID) == 0 { - // The user is not allowed to send exemplars, so there is no need to unmarshal them. - // Optimization to avoid the allocations required for unmarshaling exemplars. - req.SkipUnmarshalingExemplars = true - } + userID, err := tenant.TenantID(ctx) + if err != nil && !errors.Is(err, user.ErrNoOrgID) { // ignore user.ErrNoOrgID + return nil, nil, errors.Wrap(err, "failed to get tenant ID") + } - if err := parser(ctx, r, maxRecvMsgSize, rb, &req, logger); err != nil { - // Check for httpgrpc error, default to client error if parsing failed - if _, ok := httpgrpc.HTTPResponseFromError(err); !ok { - err = httpgrpc.Error(http.StatusBadRequest, err.Error()) - } + // userID might be empty if none was in the ctx, in this case just use the default setting. + if limits.MaxGlobalExemplarsPerUser(userID) == 0 { + // The user is not allowed to send exemplars, so there is no need to unmarshal them. + // Optimization to avoid the allocations required for unmarshaling exemplars. + req.SkipUnmarshalingExemplars = true + } - rb.CleanUp() - return nil, nil, err + if err := parser(ctx, r, maxRecvMsgSize, rb, &req, logger); err != nil { + // Check for httpgrpc error, default to client error if parsing failed + if _, ok := httpgrpc.HTTPResponseFromError(err); !ok { + err = httpgrpc.Error(http.StatusBadRequest, err.Error()) } - if allowSkipLabelNameValidation { - req.SkipLabelValidation = req.SkipLabelValidation && r.Header.Get(SkipLabelNameValidationHeader) == "true" - } else { - req.SkipLabelValidation = false - } + rb.CleanUp() + return nil, nil, err + } - if allowSkipLabelCountValidation { - req.SkipLabelCountValidation = req.SkipLabelCountValidation && r.Header.Get(SkipLabelCountValidationHeader) == "true" - } else { - req.SkipLabelCountValidation = false - } + if allowSkipLabelNameValidation { + req.SkipLabelValidation = req.SkipLabelValidation && r.Header.Get(SkipLabelNameValidationHeader) == "true" + } else { + req.SkipLabelValidation = false + } - cleanup := func() { - mimirpb.ReuseSlice(req.Timeseries) - rb.CleanUp() - } - return &req.WriteRequest, cleanup, nil + if allowSkipLabelCountValidation { + req.SkipLabelCountValidation = req.SkipLabelCountValidation && r.Header.Get(SkipLabelCountValidationHeader) == "true" + } else { + req.SkipLabelCountValidation = false + } + + cleanup := func() { + mimirpb.ReuseSlice(req.Timeseries) + rb.CleanUp() } + return &req.WriteRequest, cleanup, nil } req := newRequest(supplier) - if err := push(ctx, req); err != nil { + ctx = contextWithWriteResponseStats(ctx) + err = push(ctx, req) + rsValue := ctx.Value(PushResponseStatsContextKey) + if rsValue != nil { + rs := rsValue.(*promRemote.WriteResponseStats) + addWriteResponseStats(w, rs) + } else { + // This should not happen, but if it does, we should not panic. + addWriteResponseStats(w, &promRemote.WriteResponseStats{}) + } + if err != nil { if errors.Is(err, context.Canceled) { http.Error(w, err.Error(), statusClientClosedRequest) level.Warn(logger).Log("msg", "push request canceled", "err", err) @@ -275,6 +286,35 @@ func isRemoteWrite2(r *http.Request) (bool, error) { return false, nil } +// Consts from https://github.com/prometheus/prometheus/blob/main/storage/remote/stats.go +const ( + rw20WrittenSamplesHeader = "X-Prometheus-Remote-Write-Samples-Written" + rw20WrittenHistogramsHeader = "X-Prometheus-Remote-Write-Histograms-Written" + rw20WrittenExemplarsHeader = "X-Prometheus-Remote-Write-Exemplars-Written" +) + +func contextWithWriteResponseStats(ctx context.Context) context.Context { + return context.WithValue(ctx, PushResponseStatsContextKey, &promRemote.WriteResponseStats{}) +} + +func addWriteResponseStats(w http.ResponseWriter, rs *promRemote.WriteResponseStats) { + headers := w.Header() + headers.Add(rw20WrittenSamplesHeader, strconv.Itoa(rs.Samples)) + headers.Add(rw20WrittenHistogramsHeader, strconv.Itoa(rs.Histograms)) + headers.Add(rw20WrittenExemplarsHeader, strconv.Itoa(rs.Exemplars)) +} + +func updateWriteResponseStatsCtx(ctx context.Context, samples, histograms, exemplars int) { + prs := ctx.Value(PushResponseStatsContextKey) + if prs == nil { + // Should not happen, but we should not panic anyway. + return + } + prs.(*promRemote.WriteResponseStats).Samples += samples + prs.(*promRemote.WriteResponseStats).Histograms += histograms + prs.(*promRemote.WriteResponseStats).Exemplars += exemplars +} + func calculateRetryAfter(retryAttemptHeader string, minBackoff, maxBackoff time.Duration) string { const jitterFactor = 0.5 diff --git a/pkg/mimirpb/mimir.pb.go b/pkg/mimirpb/mimir.pb.go index 2fcc5ccde1..36005dbcae 100644 --- a/pkg/mimirpb/mimir.pb.go +++ b/pkg/mimirpb/mimir.pb.go @@ -235,6 +235,45 @@ func (QueryResponse_ErrorType) EnumDescriptor() ([]byte, []int) { return fileDescriptor_86d4d7485f544059, []int{16, 1} } +type MetadataRW2_MetricType int32 + +const ( + METRIC_TYPE_UNSPECIFIED MetadataRW2_MetricType = 0 + METRIC_TYPE_COUNTER MetadataRW2_MetricType = 1 + METRIC_TYPE_GAUGE MetadataRW2_MetricType = 2 + METRIC_TYPE_HISTOGRAM MetadataRW2_MetricType = 3 + METRIC_TYPE_GAUGEHISTOGRAM MetadataRW2_MetricType = 4 + METRIC_TYPE_SUMMARY MetadataRW2_MetricType = 5 + METRIC_TYPE_INFO MetadataRW2_MetricType = 6 + METRIC_TYPE_STATESET MetadataRW2_MetricType = 7 +) + +var MetadataRW2_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 MetadataRW2_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 (MetadataRW2_MetricType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_86d4d7485f544059, []int{27, 0} +} + type WriteRequest struct { Timeseries []PreallocTimeseries `protobuf:"bytes,1,rep,name=timeseries,proto3,customtype=PreallocTimeseries" json:"timeseries"` Source WriteRequest_SourceEnum `protobuf:"varint,2,opt,name=Source,proto3,enum=cortexpb.WriteRequest_SourceEnum" json:"Source,omitempty"` @@ -392,6 +431,12 @@ type TimeSeries struct { Samples []Sample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples"` Exemplars []Exemplar `protobuf:"bytes,3,rep,name=exemplars,proto3" json:"exemplars"` Histograms []Histogram `protobuf:"bytes,4,rep,name=histograms,proto3" json:"histograms"` + // Copy from remote write 2.0 + // 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"` // Skip unmarshaling of exemplars. SkipUnmarshalingExemplars bool @@ -450,6 +495,13 @@ func (m *TimeSeries) GetHistograms() []Histogram { return nil } +func (m *TimeSeries) GetCreatedTimestamp() int64 { + if m != nil { + return m.CreatedTimestamp + } + return 0 +} + type LabelPair struct { Name []byte `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` @@ -748,6 +800,8 @@ type Histogram struct { ResetHint Histogram_ResetHint `protobuf:"varint,14,opt,name=reset_hint,json=resetHint,proto3,enum=cortexpb.Histogram_ResetHint" json:"reset_hint,omitempty"` // timestamp is in ms format Timestamp int64 `protobuf:"varint,15,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + // Support Remote Write 2.0 + CustomValues []float64 `protobuf:"fixed64,16,rep,packed,name=custom_values,json=customValues,proto3" json:"custom_values,omitempty"` } func (m *Histogram) Reset() { *m = Histogram{} } @@ -931,6 +985,13 @@ func (m *Histogram) GetTimestamp() int64 { 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{}{ @@ -1889,6 +1950,323 @@ func (m *MatrixSeries) GetHistograms() []FloatHistogramPair { return nil } +// WriteRequest1Or2 can unmarshal both Remote Write 1.0 or 2.0 messages. +type WriteRequestRW2 struct { + Symbols []string `protobuf:"bytes,4,rep,name=symbols,proto3" json:"symbols,omitempty"` + Timeseries []TimeSeriesRW2 `protobuf:"bytes,5,rep,name=timeseries,proto3" json:"timeseries"` + // Skip validation of label names and values. + SkipLabelValidation bool `protobuf:"varint,1000,opt,name=skip_label_validation,json=skipLabelValidation,proto3" json:"skip_label_validation,omitempty"` + // Skip label count validation. + SkipLabelCountValidation bool `protobuf:"varint,1001,opt,name=skip_label_count_validation,json=skipLabelCountValidation,proto3" json:"skip_label_count_validation,omitempty"` +} + +func (m *WriteRequestRW2) Reset() { *m = WriteRequestRW2{} } +func (*WriteRequestRW2) ProtoMessage() {} +func (*WriteRequestRW2) Descriptor() ([]byte, []int) { + return fileDescriptor_86d4d7485f544059, []int{24} +} +func (m *WriteRequestRW2) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WriteRequestRW2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_WriteRequestRW2.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 *WriteRequestRW2) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteRequestRW2.Merge(m, src) +} +func (m *WriteRequestRW2) XXX_Size() int { + return m.Size() +} +func (m *WriteRequestRW2) XXX_DiscardUnknown() { + xxx_messageInfo_WriteRequestRW2.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteRequestRW2 proto.InternalMessageInfo + +func (m *WriteRequestRW2) GetSymbols() []string { + if m != nil { + return m.Symbols + } + return nil +} + +func (m *WriteRequestRW2) GetTimeseries() []TimeSeriesRW2 { + if m != nil { + return m.Timeseries + } + return nil +} + +func (m *WriteRequestRW2) GetSkipLabelValidation() bool { + if m != nil { + return m.SkipLabelValidation + } + return false +} + +func (m *WriteRequestRW2) GetSkipLabelCountValidation() bool { + if m != nil { + return m.SkipLabelCountValidation + } + return false +} + +type TimeSeriesRW2 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 []ExemplarRW2 `protobuf:"bytes,4,rep,name=exemplars,proto3" json:"exemplars"` + // metadata represents the metadata associated with the given series' samples. + Metadata MetadataRW2 `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 *TimeSeriesRW2) Reset() { *m = TimeSeriesRW2{} } +func (*TimeSeriesRW2) ProtoMessage() {} +func (*TimeSeriesRW2) Descriptor() ([]byte, []int) { + return fileDescriptor_86d4d7485f544059, []int{25} +} +func (m *TimeSeriesRW2) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TimeSeriesRW2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TimeSeriesRW2.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 *TimeSeriesRW2) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSeriesRW2.Merge(m, src) +} +func (m *TimeSeriesRW2) XXX_Size() int { + return m.Size() +} +func (m *TimeSeriesRW2) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSeriesRW2.DiscardUnknown(m) +} + +var xxx_messageInfo_TimeSeriesRW2 proto.InternalMessageInfo + +func (m *TimeSeriesRW2) GetLabelsRefs() []uint32 { + if m != nil { + return m.LabelsRefs + } + return nil +} + +func (m *TimeSeriesRW2) GetSamples() []Sample { + if m != nil { + return m.Samples + } + return nil +} + +func (m *TimeSeriesRW2) GetHistograms() []Histogram { + if m != nil { + return m.Histograms + } + return nil +} + +func (m *TimeSeriesRW2) GetExemplars() []ExemplarRW2 { + if m != nil { + return m.Exemplars + } + return nil +} + +func (m *TimeSeriesRW2) GetMetadata() MetadataRW2 { + if m != nil { + return m.Metadata + } + return MetadataRW2{} +} + +func (m *TimeSeriesRW2) GetCreatedTimestamp() int64 { + if m != nil { + return m.CreatedTimestamp + } + return 0 +} + +type ExemplarRW2 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 *ExemplarRW2) Reset() { *m = ExemplarRW2{} } +func (*ExemplarRW2) ProtoMessage() {} +func (*ExemplarRW2) Descriptor() ([]byte, []int) { + return fileDescriptor_86d4d7485f544059, []int{26} +} +func (m *ExemplarRW2) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExemplarRW2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ExemplarRW2.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 *ExemplarRW2) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExemplarRW2.Merge(m, src) +} +func (m *ExemplarRW2) XXX_Size() int { + return m.Size() +} +func (m *ExemplarRW2) XXX_DiscardUnknown() { + xxx_messageInfo_ExemplarRW2.DiscardUnknown(m) +} + +var xxx_messageInfo_ExemplarRW2 proto.InternalMessageInfo + +func (m *ExemplarRW2) GetLabelsRefs() []uint32 { + if m != nil { + return m.LabelsRefs + } + return nil +} + +func (m *ExemplarRW2) GetValue() float64 { + if m != nil { + return m.Value + } + return 0 +} + +func (m *ExemplarRW2) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +type MetadataRW2 struct { + Type MetadataRW2_MetricType `protobuf:"varint,1,opt,name=type,proto3,enum=cortexpb.MetadataRW2_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 *MetadataRW2) Reset() { *m = MetadataRW2{} } +func (*MetadataRW2) ProtoMessage() {} +func (*MetadataRW2) Descriptor() ([]byte, []int) { + return fileDescriptor_86d4d7485f544059, []int{27} +} +func (m *MetadataRW2) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MetadataRW2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_MetadataRW2.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 *MetadataRW2) XXX_Merge(src proto.Message) { + xxx_messageInfo_MetadataRW2.Merge(m, src) +} +func (m *MetadataRW2) XXX_Size() int { + return m.Size() +} +func (m *MetadataRW2) XXX_DiscardUnknown() { + xxx_messageInfo_MetadataRW2.DiscardUnknown(m) +} + +var xxx_messageInfo_MetadataRW2 proto.InternalMessageInfo + +func (m *MetadataRW2) GetType() MetadataRW2_MetricType { + if m != nil { + return m.Type + } + return METRIC_TYPE_UNSPECIFIED +} + +func (m *MetadataRW2) GetHelpRef() uint32 { + if m != nil { + return m.HelpRef + } + return 0 +} + +func (m *MetadataRW2) GetUnitRef() uint32 { + if m != nil { + return m.UnitRef + } + return 0 +} + func init() { proto.RegisterEnum("cortexpb.ErrorCause", ErrorCause_name, ErrorCause_value) proto.RegisterEnum("cortexpb.WriteRequest_SourceEnum", WriteRequest_SourceEnum_name, WriteRequest_SourceEnum_value) @@ -1896,6 +2274,7 @@ func init() { proto.RegisterEnum("cortexpb.Histogram_ResetHint", Histogram_ResetHint_name, Histogram_ResetHint_value) proto.RegisterEnum("cortexpb.QueryResponse_Status", QueryResponse_Status_name, QueryResponse_Status_value) proto.RegisterEnum("cortexpb.QueryResponse_ErrorType", QueryResponse_ErrorType_name, QueryResponse_ErrorType_value) + proto.RegisterEnum("cortexpb.MetadataRW2_MetricType", MetadataRW2_MetricType_name, MetadataRW2_MetricType_value) proto.RegisterType((*WriteRequest)(nil), "cortexpb.WriteRequest") proto.RegisterType((*WriteResponse)(nil), "cortexpb.WriteResponse") proto.RegisterType((*ErrorDetails)(nil), "cortexpb.ErrorDetails") @@ -1920,139 +2299,161 @@ func init() { proto.RegisterType((*ScalarData)(nil), "cortexpb.ScalarData") proto.RegisterType((*MatrixData)(nil), "cortexpb.MatrixData") proto.RegisterType((*MatrixSeries)(nil), "cortexpb.MatrixSeries") + proto.RegisterType((*WriteRequestRW2)(nil), "cortexpb.WriteRequestRW2") + proto.RegisterType((*TimeSeriesRW2)(nil), "cortexpb.TimeSeriesRW2") + proto.RegisterType((*ExemplarRW2)(nil), "cortexpb.ExemplarRW2") + proto.RegisterType((*MetadataRW2)(nil), "cortexpb.MetadataRW2") } func init() { proto.RegisterFile("mimir.proto", fileDescriptor_86d4d7485f544059) } var fileDescriptor_86d4d7485f544059 = []byte{ - // 2029 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcf, 0x93, 0xdb, 0x48, - 0xf5, 0xb7, 0x6c, 0xf9, 0x87, 0xde, 0xd8, 0x33, 0x9d, 0x4e, 0x36, 0x5f, 0x6f, 0xbe, 0x1b, 0x27, - 0xd1, 0x16, 0xcb, 0x90, 0x82, 0x09, 0xb5, 0x81, 0x6c, 0x6d, 0x2a, 0xfc, 0x90, 0x6d, 0x25, 0xe3, - 0xc4, 0x96, 0x67, 0x5b, 0xf2, 0x84, 0x70, 0x51, 0x69, 0x3c, 0x3d, 0x33, 0xaa, 0xb5, 0x2c, 0x23, - 0xc9, 0xd9, 0x0c, 0x27, 0x2e, 0x50, 0x14, 0x27, 0x2e, 0x5c, 0x28, 0x6e, 0x5c, 0xa8, 0xe2, 0x1f, - 0x49, 0x15, 0x97, 0x1c, 0x17, 0x0e, 0x29, 0x32, 0xb9, 0x2c, 0x07, 0xaa, 0xb6, 0x28, 0x4e, 0x9c, - 0xa8, 0xee, 0xd6, 0x4f, 0xcf, 0x0c, 0x04, 0xc8, 0x4d, 0xef, 0xbd, 0xcf, 0x7b, 0xfd, 0xba, 0xdf, - 0x8f, 0x7e, 0x2d, 0x58, 0xf3, 0x5c, 0xcf, 0x0d, 0xb6, 0x16, 0x81, 0x1f, 0xf9, 0xb8, 0x31, 0xf5, - 0x83, 0x88, 0x3e, 0x5b, 0xec, 0x5d, 0xf9, 0xc6, 0xa1, 0x1b, 0x1d, 0x2d, 0xf7, 0xb6, 0xa6, 0xbe, - 0x77, 0xeb, 0xd0, 0x3f, 0xf4, 0x6f, 0x71, 0xc0, 0xde, 0xf2, 0x80, 0x53, 0x9c, 0xe0, 0x5f, 0x42, - 0x51, 0xfd, 0x6b, 0x19, 0x9a, 0x8f, 0x03, 0x37, 0xa2, 0x84, 0xfe, 0x68, 0x49, 0xc3, 0x08, 0xef, - 0x00, 0x44, 0xae, 0x47, 0x43, 0x1a, 0xb8, 0x34, 0x6c, 0x4b, 0xd7, 0x2b, 0x9b, 0x6b, 0x1f, 0x5e, - 0xda, 0x4a, 0xcc, 0x6f, 0x59, 0xae, 0x47, 0x4d, 0x2e, 0xeb, 0x5e, 0x79, 0xfe, 0xf2, 0x5a, 0xe9, - 0x4f, 0x2f, 0xaf, 0xe1, 0x9d, 0x80, 0x3a, 0xb3, 0x99, 0x3f, 0xb5, 0x52, 0x3d, 0x92, 0xb3, 0x81, - 0x3f, 0x86, 0x9a, 0xe9, 0x2f, 0x83, 0x29, 0x6d, 0x97, 0xaf, 0x4b, 0x9b, 0xeb, 0x1f, 0xde, 0xc8, - 0xac, 0xe5, 0x57, 0xde, 0x12, 0x20, 0x7d, 0xbe, 0xf4, 0x48, 0xac, 0x80, 0xef, 0x42, 0xc3, 0xa3, - 0x91, 0xb3, 0xef, 0x44, 0x4e, 0xbb, 0xc2, 0x5d, 0x69, 0x67, 0xca, 0x23, 0x1a, 0x05, 0xee, 0x74, - 0x14, 0xcb, 0xbb, 0xf2, 0xf3, 0x97, 0xd7, 0x24, 0x92, 0xe2, 0xf1, 0x6d, 0x78, 0x27, 0xfc, 0xd4, - 0x5d, 0xd8, 0x33, 0x67, 0x8f, 0xce, 0xec, 0xa7, 0xce, 0xcc, 0xdd, 0x77, 0x22, 0xd7, 0x9f, 0xb7, - 0xbf, 0xa8, 0x5f, 0x97, 0x36, 0x1b, 0xe4, 0x22, 0x93, 0x0e, 0x99, 0x70, 0x37, 0x95, 0xe1, 0xef, - 0xc2, 0xff, 0xe7, 0x94, 0xa6, 0xfe, 0x72, 0x1e, 0xe5, 0x55, 0xff, 0x22, 0x54, 0xdb, 0xa9, 0x6a, - 0x8f, 0x21, 0x32, 0x7d, 0xf5, 0x1a, 0x40, 0xb6, 0x0d, 0x5c, 0x87, 0x8a, 0xb6, 0x33, 0x40, 0x25, - 0xdc, 0x00, 0x99, 0x4c, 0x86, 0x3a, 0x92, 0xd4, 0x0d, 0x68, 0xc5, 0x9b, 0x0e, 0x17, 0xfe, 0x3c, - 0xa4, 0xea, 0x5d, 0x68, 0xea, 0x41, 0xe0, 0x07, 0x7d, 0x1a, 0x39, 0xee, 0x2c, 0xc4, 0x37, 0xa1, - 0xda, 0x73, 0x96, 0x21, 0x6d, 0x4b, 0xfc, 0xb0, 0x72, 0x47, 0xcf, 0x61, 0x5c, 0x46, 0x04, 0x44, - 0xfd, 0xbb, 0x04, 0x90, 0x05, 0x04, 0x6b, 0x50, 0xe3, 0x7e, 0x27, 0x61, 0xbb, 0x98, 0xe9, 0x72, - 0x67, 0x77, 0x1c, 0x37, 0xe8, 0x5e, 0x8a, 0xa3, 0xd6, 0xe4, 0x2c, 0x6d, 0xdf, 0x59, 0x44, 0x34, - 0x20, 0xb1, 0x22, 0xfe, 0x26, 0xd4, 0x43, 0xc7, 0x5b, 0xcc, 0x68, 0xd8, 0x2e, 0x73, 0x1b, 0x28, - 0xb3, 0x61, 0x72, 0x01, 0x3f, 0xe7, 0x12, 0x49, 0x60, 0xf8, 0x0e, 0x28, 0xf4, 0x19, 0xf5, 0x16, - 0x33, 0x27, 0x08, 0xe3, 0x18, 0xe1, 0x9c, 0xcf, 0xb1, 0x28, 0xd6, 0xca, 0xa0, 0xf8, 0x63, 0x80, - 0x23, 0x37, 0x8c, 0xfc, 0xc3, 0xc0, 0xf1, 0xc2, 0xb6, 0xbc, 0xea, 0xf0, 0x76, 0x22, 0x8b, 0x35, - 0x73, 0x60, 0xf5, 0xdb, 0xa0, 0xa4, 0xfb, 0xc1, 0x18, 0xe4, 0xb9, 0xe3, 0x89, 0xe3, 0x6a, 0x12, - 0xfe, 0x8d, 0x2f, 0x41, 0xf5, 0xa9, 0x33, 0x5b, 0x8a, 0x84, 0x6b, 0x12, 0x41, 0xa8, 0x1a, 0xd4, - 0xc4, 0x16, 0xf0, 0x0d, 0x68, 0xf2, 0xfc, 0x8c, 0x1c, 0x6f, 0x61, 0x7b, 0x21, 0x87, 0x55, 0xc8, - 0x5a, 0xca, 0x1b, 0x85, 0x99, 0x09, 0x66, 0x57, 0x4a, 0x4c, 0xfc, 0xba, 0x0c, 0xeb, 0xc5, 0xb4, - 0xc3, 0x1f, 0x81, 0x1c, 0x1d, 0x2f, 0x92, 0x70, 0xbd, 0x7f, 0x5e, 0x7a, 0xc6, 0xa4, 0x75, 0xbc, - 0xa0, 0x84, 0x2b, 0xe0, 0xaf, 0x03, 0xf6, 0x38, 0xcf, 0x3e, 0x70, 0x3c, 0x77, 0x76, 0x6c, 0xf3, - 0x6d, 0x30, 0x57, 0x14, 0x82, 0x84, 0xe4, 0x3e, 0x17, 0x18, 0x6c, 0x4b, 0x18, 0xe4, 0x23, 0x3a, - 0x5b, 0xb4, 0x65, 0x2e, 0xe7, 0xdf, 0x8c, 0xb7, 0x9c, 0xbb, 0x51, 0xbb, 0x2a, 0x78, 0xec, 0x5b, - 0x3d, 0x06, 0xc8, 0x56, 0xc2, 0x6b, 0x50, 0x9f, 0x18, 0x8f, 0x8c, 0xf1, 0x63, 0x03, 0x95, 0x18, - 0xd1, 0x1b, 0x4f, 0x0c, 0x4b, 0x27, 0x48, 0xc2, 0x0a, 0x54, 0x1f, 0x68, 0x93, 0x07, 0x3a, 0x2a, - 0xe3, 0x16, 0x28, 0xdb, 0x03, 0xd3, 0x1a, 0x3f, 0x20, 0xda, 0x08, 0x55, 0x30, 0x86, 0x75, 0x2e, - 0xc9, 0x78, 0x32, 0x53, 0x35, 0x27, 0xa3, 0x91, 0x46, 0x9e, 0xa0, 0x2a, 0x4b, 0xe6, 0x81, 0x71, - 0x7f, 0x8c, 0x6a, 0xb8, 0x09, 0x0d, 0xd3, 0xd2, 0x2c, 0xdd, 0xd4, 0x2d, 0x54, 0x57, 0x1f, 0x41, - 0x4d, 0x2c, 0xfd, 0x16, 0x12, 0x51, 0xfd, 0x99, 0x04, 0x8d, 0x24, 0x79, 0xde, 0x46, 0x62, 0x17, - 0x52, 0x22, 0x89, 0xe7, 0xa9, 0x44, 0xa8, 0x9c, 0x4a, 0x04, 0xf5, 0x0f, 0x55, 0x50, 0xd2, 0x64, - 0xc4, 0x57, 0x41, 0x11, 0x4d, 0xc1, 0x9d, 0x47, 0x3c, 0xe4, 0xf2, 0x76, 0x89, 0x34, 0x38, 0x6b, - 0x30, 0x8f, 0xf0, 0x0d, 0x58, 0x13, 0xe2, 0x83, 0x99, 0xef, 0x44, 0x62, 0xad, 0xed, 0x12, 0x01, - 0xce, 0xbc, 0xcf, 0x78, 0x18, 0x41, 0x25, 0x5c, 0x7a, 0x7c, 0x25, 0x89, 0xb0, 0x4f, 0x7c, 0x19, - 0x6a, 0xe1, 0xf4, 0x88, 0x7a, 0x0e, 0x0f, 0xee, 0x05, 0x12, 0x53, 0xf8, 0x2b, 0xb0, 0xfe, 0x63, - 0x1a, 0xf8, 0x76, 0x74, 0x14, 0xd0, 0xf0, 0xc8, 0x9f, 0xed, 0xf3, 0x40, 0x4b, 0xa4, 0xc5, 0xb8, - 0x56, 0xc2, 0xc4, 0x1f, 0xc4, 0xb0, 0xcc, 0xaf, 0x1a, 0xf7, 0x4b, 0x22, 0x4d, 0xc6, 0xef, 0x25, - 0xbe, 0xdd, 0x04, 0x94, 0xc3, 0x09, 0x07, 0xeb, 0xdc, 0x41, 0x89, 0xac, 0xa7, 0x48, 0xe1, 0xa4, - 0x06, 0xeb, 0x73, 0x7a, 0xe8, 0x44, 0xee, 0x53, 0x6a, 0x87, 0x0b, 0x67, 0x1e, 0xb6, 0x1b, 0xab, - 0x17, 0x41, 0x77, 0x39, 0xfd, 0x94, 0x46, 0xe6, 0xc2, 0x99, 0xc7, 0x15, 0xda, 0x4a, 0x34, 0x18, - 0x2f, 0xc4, 0x5f, 0x85, 0x8d, 0xd4, 0xc4, 0x3e, 0x9d, 0x45, 0x4e, 0xd8, 0x56, 0xae, 0x57, 0x36, - 0x31, 0x49, 0x2d, 0xf7, 0x39, 0xb7, 0x00, 0xe4, 0xbe, 0x85, 0x6d, 0xb8, 0x5e, 0xd9, 0x94, 0x32, - 0x20, 0x77, 0x8c, 0xb5, 0xb7, 0xf5, 0x85, 0x1f, 0xba, 0x39, 0xa7, 0xd6, 0xfe, 0xbd, 0x53, 0x89, - 0x46, 0xea, 0x54, 0x6a, 0x22, 0x76, 0xaa, 0x29, 0x9c, 0x4a, 0xd8, 0x99, 0x53, 0x29, 0x30, 0x76, - 0xaa, 0x25, 0x9c, 0x4a, 0xd8, 0xb1, 0x53, 0xf7, 0x00, 0x02, 0x1a, 0xd2, 0xc8, 0x3e, 0x62, 0x27, - 0xbf, 0xce, 0x9b, 0xc0, 0xd5, 0x33, 0xda, 0xd8, 0x16, 0x61, 0xa8, 0x6d, 0x77, 0x1e, 0x11, 0x25, - 0x48, 0x3e, 0xf1, 0x7b, 0xa0, 0xa4, 0xb9, 0xd6, 0xde, 0xe0, 0xc9, 0x97, 0x31, 0xd4, 0xbb, 0xa0, - 0xa4, 0x5a, 0xc5, 0x52, 0xae, 0x43, 0xe5, 0x89, 0x6e, 0x22, 0x09, 0xd7, 0xa0, 0x6c, 0x8c, 0x51, - 0x39, 0x2b, 0xe7, 0xca, 0x15, 0xf9, 0xe7, 0xbf, 0xed, 0x48, 0xdd, 0x3a, 0x54, 0xb9, 0xdf, 0xdd, - 0x26, 0x40, 0x16, 0x76, 0xf5, 0x6f, 0x32, 0xac, 0xf3, 0x10, 0x67, 0x29, 0x1d, 0x02, 0xe6, 0x32, - 0x1a, 0xd8, 0x2b, 0x3b, 0x69, 0x75, 0xf5, 0x7f, 0xbc, 0xbc, 0xa6, 0xe5, 0x06, 0x8a, 0x45, 0xe0, - 0x7b, 0x34, 0x3a, 0xa2, 0xcb, 0x30, 0xff, 0xe9, 0xf9, 0xfb, 0x74, 0x76, 0x2b, 0x6d, 0xd0, 0x5b, - 0x3d, 0x61, 0x2e, 0xdb, 0x31, 0x9a, 0xae, 0x70, 0xfe, 0xd7, 0x9c, 0xbf, 0x9a, 0xdf, 0x94, 0xc8, - 0x62, 0xa2, 0xa4, 0x39, 0xcc, 0x8a, 0x5d, 0x48, 0xe2, 0x62, 0xe7, 0xc4, 0x19, 0x95, 0xf7, 0x16, - 0x32, 0xea, 0x2d, 0x54, 0xca, 0xd7, 0x00, 0xa5, 0x5e, 0xec, 0x71, 0x6c, 0x92, 0x6c, 0x69, 0x0e, - 0x0a, 0x13, 0x1c, 0x9a, 0xae, 0x96, 0x40, 0x45, 0xb1, 0xa4, 0x35, 0x94, 0x40, 0xdf, 0x87, 0xd6, - 0x74, 0x19, 0x46, 0xbe, 0x67, 0xf3, 0x56, 0x17, 0xb6, 0x11, 0xc7, 0x35, 0x05, 0x73, 0x97, 0xf3, - 0x1e, 0xca, 0x0d, 0x09, 0x95, 0x1f, 0xca, 0x8d, 0x1a, 0xaa, 0x3f, 0x94, 0x1b, 0x0a, 0x82, 0x87, - 0x72, 0xa3, 0x89, 0x5a, 0x0f, 0xe5, 0xc6, 0x06, 0x42, 0x24, 0x6b, 0x75, 0x64, 0xa5, 0xc5, 0x90, - 0xd5, 0xda, 0x26, 0xab, 0x75, 0x95, 0xcf, 0xe3, 0x7b, 0x00, 0xd9, 0x19, 0xb0, 0xd0, 0xfb, 0x07, - 0x07, 0x21, 0x15, 0xfd, 0xf3, 0x02, 0x89, 0x29, 0xc6, 0x9f, 0xd1, 0xf9, 0x61, 0x74, 0xc4, 0xa3, - 0xd6, 0x22, 0x31, 0xa5, 0x2e, 0x01, 0x17, 0x33, 0x96, 0x5f, 0xfb, 0x6f, 0x70, 0x85, 0xdf, 0x03, - 0x25, 0xcd, 0x49, 0xbe, 0x56, 0x61, 0x7a, 0x2c, 0xda, 0x8c, 0xa7, 0xc7, 0x4c, 0x41, 0x9d, 0xc3, - 0x86, 0x98, 0x16, 0xb2, 0x4a, 0x49, 0xd3, 0x4a, 0x3a, 0x23, 0xad, 0xca, 0x59, 0x5a, 0xdd, 0x86, - 0x7a, 0x12, 0x1c, 0x31, 0x10, 0xbd, 0x7b, 0xd6, 0x5c, 0xc3, 0x11, 0x24, 0x41, 0xaa, 0x21, 0x6c, - 0xac, 0xc8, 0x70, 0x07, 0x60, 0xcf, 0x5f, 0xce, 0xf7, 0x9d, 0x78, 0x14, 0x97, 0x36, 0xab, 0x24, - 0xc7, 0x61, 0xfe, 0xcc, 0xfc, 0xcf, 0x68, 0x90, 0xa4, 0x39, 0x27, 0x18, 0x77, 0xb9, 0x58, 0xd0, - 0x20, 0x4e, 0x74, 0x41, 0x64, 0xbe, 0xcb, 0x39, 0xdf, 0xd5, 0x19, 0x5c, 0x5c, 0xd9, 0x24, 0x3f, - 0xdc, 0x42, 0x5b, 0x2a, 0xaf, 0xb4, 0x25, 0xfc, 0xd1, 0xe9, 0x73, 0x7d, 0x77, 0x75, 0x4a, 0x4c, - 0xed, 0xe5, 0x8f, 0xf4, 0x8f, 0x32, 0xb4, 0x3e, 0x59, 0xd2, 0xe0, 0x38, 0x19, 0x7e, 0xf1, 0x1d, - 0xa8, 0x85, 0x91, 0x13, 0x2d, 0xc3, 0x78, 0x7c, 0xea, 0x64, 0x76, 0x0a, 0xc0, 0x2d, 0x93, 0xa3, - 0x48, 0x8c, 0xc6, 0xdf, 0x07, 0xa0, 0x6c, 0x1a, 0xb6, 0xf9, 0xe8, 0x75, 0xea, 0x59, 0x51, 0xd4, - 0xe5, 0x73, 0x33, 0x1f, 0xbc, 0x14, 0x9a, 0x7c, 0xb2, 0xf3, 0xe0, 0x04, 0x3f, 0x25, 0x85, 0x08, - 0x02, 0x6f, 0x31, 0x7f, 0x02, 0x77, 0x7e, 0xc8, 0x8f, 0xa9, 0x50, 0xc5, 0x26, 0xe7, 0xf7, 0x9d, - 0xc8, 0xd9, 0x2e, 0x91, 0x18, 0xc5, 0xf0, 0x4f, 0xe9, 0x34, 0xf2, 0x03, 0xde, 0xa6, 0x0a, 0xf8, - 0x5d, 0xce, 0x4f, 0xf0, 0x02, 0xc5, 0xed, 0x4f, 0x9d, 0x99, 0x13, 0xf0, 0x3b, 0xba, 0x68, 0x9f, - 0xf3, 0x53, 0xfb, 0x9c, 0x62, 0x78, 0xcf, 0x89, 0x02, 0xf7, 0x19, 0xef, 0x71, 0x05, 0xfc, 0x88, - 0xf3, 0x13, 0xbc, 0x40, 0xe1, 0x2b, 0xd0, 0xf8, 0xcc, 0x09, 0xe6, 0xee, 0xfc, 0x50, 0xf4, 0x21, - 0x85, 0xa4, 0x34, 0xdb, 0xb1, 0x3b, 0x3f, 0xf0, 0xc5, 0x35, 0xac, 0x10, 0x41, 0xa8, 0x1f, 0x40, - 0x4d, 0x9c, 0x2d, 0xbb, 0x42, 0x74, 0x42, 0xc6, 0x44, 0x4c, 0x8a, 0xe6, 0xa4, 0xd7, 0xd3, 0x4d, - 0x13, 0x49, 0xe2, 0x3e, 0x51, 0x7f, 0x25, 0x81, 0x92, 0x1e, 0x24, 0x1b, 0x01, 0x8d, 0xb1, 0xa1, - 0x0b, 0xa8, 0x35, 0x18, 0xe9, 0xe3, 0x89, 0x85, 0x24, 0x36, 0x0f, 0xf6, 0x34, 0xa3, 0xa7, 0x0f, - 0xf5, 0xbe, 0x98, 0x2b, 0xf5, 0x1f, 0xe8, 0xbd, 0x89, 0x35, 0x18, 0x1b, 0xa8, 0xc2, 0x84, 0x5d, - 0xad, 0x6f, 0xf7, 0x35, 0x4b, 0x43, 0x32, 0xa3, 0x06, 0x6c, 0x14, 0x35, 0xb4, 0x21, 0xaa, 0xe2, - 0x0d, 0x58, 0x9b, 0x18, 0xda, 0xae, 0x36, 0x18, 0x6a, 0xdd, 0xa1, 0x8e, 0x6a, 0x4c, 0xd7, 0x18, - 0x5b, 0xf6, 0xfd, 0xf1, 0xc4, 0xe8, 0xa3, 0x3a, 0x9b, 0x49, 0x19, 0xa9, 0xf5, 0x7a, 0xfa, 0x8e, - 0xc5, 0x21, 0x8d, 0xf8, 0x9e, 0xab, 0x81, 0xcc, 0xc6, 0x6b, 0x55, 0x07, 0xc8, 0x22, 0x54, 0x9c, - 0xde, 0x95, 0xf3, 0xa6, 0xbd, 0xd3, 0x3d, 0x43, 0xfd, 0xa9, 0x04, 0x90, 0x45, 0x0e, 0xdf, 0xc9, - 0x9e, 0x43, 0x62, 0xf2, 0xbc, 0xbc, 0x1a, 0xe0, 0xb3, 0x1f, 0x45, 0xdf, 0x2b, 0x3c, 0x6e, 0xca, - 0xab, 0x4d, 0x40, 0xa8, 0xfe, 0xab, 0x27, 0x8e, 0x0d, 0xcd, 0xbc, 0x7d, 0xd6, 0x1c, 0xc5, 0x93, - 0x80, 0xfb, 0xa1, 0x90, 0x98, 0xfa, 0xef, 0xc7, 0xda, 0x5f, 0x48, 0xb0, 0xb1, 0xe2, 0xc6, 0xb9, - 0x8b, 0x14, 0x1a, 0x69, 0xf9, 0x0d, 0x1a, 0x69, 0x29, 0x57, 0xf5, 0x6f, 0xe2, 0x0c, 0x0b, 0x5e, - 0x9a, 0xfe, 0x67, 0x3f, 0xbd, 0xde, 0x24, 0x78, 0x5d, 0x80, 0xac, 0x2a, 0xf0, 0xb7, 0xa0, 0x56, - 0xf8, 0x89, 0x71, 0x79, 0xb5, 0x76, 0xe2, 0xdf, 0x18, 0xc2, 0xe1, 0x18, 0xab, 0xfe, 0x46, 0x82, - 0x66, 0x5e, 0x7c, 0xee, 0xa1, 0xfc, 0xe7, 0x2f, 0xe5, 0x6e, 0x21, 0x29, 0xc4, 0xcd, 0xf0, 0xde, - 0x79, 0xe7, 0xc8, 0x9f, 0x34, 0xa7, 0xf2, 0xe2, 0xe6, 0xef, 0xcb, 0x00, 0xd9, 0x7f, 0x00, 0x7c, - 0x01, 0x5a, 0xf1, 0x50, 0x68, 0xf7, 0xb4, 0x89, 0xc9, 0x0a, 0xf2, 0x0a, 0x5c, 0x26, 0xfa, 0xce, - 0x70, 0xd0, 0xd3, 0x4c, 0xbb, 0x3f, 0xe8, 0xdb, 0xac, 0x6e, 0x46, 0x9a, 0xd5, 0xdb, 0x46, 0x12, - 0x7e, 0x07, 0x2e, 0x58, 0xe3, 0xb1, 0x3d, 0xd2, 0x8c, 0x27, 0x76, 0x6f, 0x38, 0x31, 0x2d, 0x9d, - 0x98, 0xa8, 0x5c, 0xa8, 0xcc, 0x0a, 0x33, 0x30, 0x30, 0x1e, 0xe8, 0x26, 0x2b, 0x5b, 0x9b, 0x68, - 0x96, 0x6e, 0x0f, 0x07, 0xa3, 0x81, 0xa5, 0xf7, 0x91, 0x8c, 0xdb, 0x70, 0x89, 0xe8, 0x9f, 0x4c, - 0x74, 0xd3, 0x2a, 0x4a, 0xaa, 0xac, 0x42, 0x07, 0x86, 0x69, 0xb1, 0xea, 0x17, 0x5c, 0x54, 0xc3, - 0xff, 0x07, 0x17, 0x4d, 0x9d, 0xec, 0x0e, 0x7a, 0xba, 0x9d, 0xaf, 0xee, 0x3a, 0xbe, 0x04, 0xc8, - 0x32, 0xfb, 0xdd, 0x02, 0xb7, 0xc1, 0xdc, 0x60, 0xde, 0x75, 0x27, 0xe6, 0x13, 0xa4, 0xb0, 0xa5, - 0x7a, 0x03, 0xd2, 0x9b, 0x0c, 0x2c, 0xbb, 0x4b, 0x74, 0xed, 0x91, 0x4e, 0xec, 0xf1, 0x8e, 0x6e, - 0x20, 0xc0, 0x97, 0x01, 0x8f, 0x74, 0x6b, 0x7b, 0x2c, 0xf6, 0xa6, 0x0d, 0x87, 0xe3, 0xc7, 0x7a, - 0x1f, 0xad, 0x61, 0x04, 0x4d, 0x4b, 0x37, 0x34, 0xc3, 0x8a, 0x1d, 0x68, 0x76, 0xbf, 0xf3, 0xe2, - 0x55, 0xa7, 0xf4, 0xf9, 0xab, 0x4e, 0xe9, 0xcb, 0x57, 0x1d, 0xe9, 0x27, 0x27, 0x1d, 0xe9, 0x77, - 0x27, 0x1d, 0xe9, 0xf9, 0x49, 0x47, 0x7a, 0x71, 0xd2, 0x91, 0xfe, 0x7c, 0xd2, 0x91, 0xbe, 0x38, - 0xe9, 0x94, 0xbe, 0x3c, 0xe9, 0x48, 0xbf, 0x7c, 0xdd, 0x29, 0xbd, 0x78, 0xdd, 0x29, 0x7d, 0xfe, - 0xba, 0x53, 0xfa, 0x61, 0x9d, 0xff, 0x58, 0x5b, 0xec, 0xed, 0xd5, 0xf8, 0x2f, 0xb2, 0xdb, 0xff, - 0x0c, 0x00, 0x00, 0xff, 0xff, 0xc3, 0xd5, 0x81, 0xca, 0x6a, 0x13, 0x00, 0x00, + // 2309 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x58, 0xcf, 0x6f, 0xdb, 0xc8, + 0xf5, 0x17, 0x25, 0x5a, 0x12, 0x9f, 0x24, 0x9b, 0x99, 0xfc, 0x58, 0xc5, 0xbb, 0x91, 0x1d, 0x2e, + 0xbe, 0xfb, 0x75, 0xd3, 0xd6, 0x29, 0x92, 0x6d, 0x82, 0x04, 0xd9, 0xb6, 0x94, 0xc4, 0xc4, 0x4a, + 0x24, 0xca, 0x3b, 0xa4, 0x92, 0xa6, 0x17, 0x82, 0x96, 0xc7, 0x36, 0xb1, 0xa2, 0xa8, 0x92, 0x54, + 0x36, 0xee, 0xa9, 0x97, 0x16, 0x45, 0x4f, 0xbd, 0xf4, 0x52, 0xf4, 0x52, 0xf4, 0x52, 0xa0, 0xff, + 0x43, 0xcf, 0x39, 0xe6, 0xb8, 0x5b, 0xa0, 0x69, 0xe3, 0x5c, 0xb6, 0x87, 0x02, 0x41, 0x8f, 0x3d, + 0x15, 0x33, 0xc3, 0x9f, 0xb2, 0xd3, 0xf5, 0x6e, 0x03, 0xf4, 0xc6, 0xf7, 0x73, 0x3e, 0xf3, 0xe6, + 0xcd, 0x9b, 0xc7, 0x07, 0x35, 0xd7, 0x71, 0x1d, 0x7f, 0x73, 0xe6, 0x7b, 0xa1, 0x87, 0xaa, 0x63, + 0xcf, 0x0f, 0xc9, 0xd3, 0xd9, 0xce, 0xea, 0xb7, 0xf7, 0x9d, 0xf0, 0x60, 0xbe, 0xb3, 0x39, 0xf6, + 0xdc, 0xab, 0xfb, 0xde, 0xbe, 0x77, 0x95, 0x29, 0xec, 0xcc, 0xf7, 0x18, 0xc5, 0x08, 0xf6, 0xc5, + 0x0d, 0x95, 0x7f, 0x14, 0xa1, 0xfe, 0xc8, 0x77, 0x42, 0x82, 0xc9, 0x8f, 0xe7, 0x24, 0x08, 0xd1, + 0x36, 0x40, 0xe8, 0xb8, 0x24, 0x20, 0xbe, 0x43, 0x82, 0xa6, 0xb0, 0x5e, 0xda, 0xa8, 0x5d, 0x3b, + 0xb7, 0x19, 0xbb, 0xdf, 0x34, 0x1d, 0x97, 0x18, 0x4c, 0xd6, 0x5e, 0x7d, 0xf6, 0x62, 0xad, 0xf0, + 0xe7, 0x17, 0x6b, 0x68, 0xdb, 0x27, 0xf6, 0x64, 0xe2, 0x8d, 0xcd, 0xc4, 0x0e, 0x67, 0x7c, 0xa0, + 0x5b, 0x50, 0x36, 0xbc, 0xb9, 0x3f, 0x26, 0xcd, 0xe2, 0xba, 0xb0, 0xb1, 0x7c, 0xed, 0x72, 0xea, + 0x2d, 0xbb, 0xf2, 0x26, 0x57, 0xd2, 0xa6, 0x73, 0x17, 0x47, 0x06, 0xe8, 0x36, 0x54, 0x5d, 0x12, + 0xda, 0xbb, 0x76, 0x68, 0x37, 0x4b, 0x0c, 0x4a, 0x33, 0x35, 0x1e, 0x90, 0xd0, 0x77, 0xc6, 0x83, + 0x48, 0xde, 0x16, 0x9f, 0xbd, 0x58, 0x13, 0x70, 0xa2, 0x8f, 0xae, 0xc3, 0xf9, 0xe0, 0x13, 0x67, + 0x66, 0x4d, 0xec, 0x1d, 0x32, 0xb1, 0x9e, 0xd8, 0x13, 0x67, 0xd7, 0x0e, 0x1d, 0x6f, 0xda, 0xfc, + 0xa2, 0xb2, 0x2e, 0x6c, 0x54, 0xf1, 0x59, 0x2a, 0xed, 0x53, 0xe1, 0xc3, 0x44, 0x86, 0xbe, 0x07, + 0xef, 0x66, 0x8c, 0xc6, 0xde, 0x7c, 0x1a, 0x66, 0x4d, 0xff, 0xce, 0x4d, 0x9b, 0x89, 0x69, 0x87, + 0x6a, 0xa4, 0xf6, 0xca, 0x1a, 0x40, 0xba, 0x0d, 0x54, 0x81, 0x92, 0xba, 0xdd, 0x93, 0x0b, 0xa8, + 0x0a, 0x22, 0x1e, 0xf5, 0x35, 0x59, 0x50, 0x56, 0xa0, 0x11, 0x6d, 0x3a, 0x98, 0x79, 0xd3, 0x80, + 0x28, 0xb7, 0xa1, 0xae, 0xf9, 0xbe, 0xe7, 0x77, 0x49, 0x68, 0x3b, 0x93, 0x00, 0x5d, 0x81, 0xa5, + 0x8e, 0x3d, 0x0f, 0x48, 0x53, 0x60, 0xc1, 0xca, 0x84, 0x9e, 0xa9, 0x31, 0x19, 0xe6, 0x2a, 0xca, + 0xef, 0x8a, 0x00, 0xe9, 0x81, 0x20, 0x15, 0xca, 0x0c, 0x77, 0x7c, 0x6c, 0x67, 0x53, 0x5b, 0x06, + 0x76, 0xdb, 0x76, 0xfc, 0xf6, 0xb9, 0xe8, 0xd4, 0xea, 0x8c, 0xa5, 0xee, 0xda, 0xb3, 0x90, 0xf8, + 0x38, 0x32, 0x44, 0xdf, 0x81, 0x4a, 0x60, 0xbb, 0xb3, 0x09, 0x09, 0x9a, 0x45, 0xe6, 0x43, 0x4e, + 0x7d, 0x18, 0x4c, 0xc0, 0xe2, 0x5c, 0xc0, 0xb1, 0x1a, 0xba, 0x01, 0x12, 0x79, 0x4a, 0xdc, 0xd9, + 0xc4, 0xf6, 0x83, 0xe8, 0x8c, 0x50, 0x06, 0x73, 0x24, 0x8a, 0xac, 0x52, 0x55, 0x74, 0x0b, 0xe0, + 0xc0, 0x09, 0x42, 0x6f, 0xdf, 0xb7, 0xdd, 0xa0, 0x29, 0x2e, 0x02, 0xde, 0x8a, 0x65, 0x91, 0x65, + 0x46, 0x19, 0x7d, 0x13, 0xce, 0x8c, 0x7d, 0x62, 0x87, 0x64, 0xd7, 0x62, 0x69, 0x16, 0xda, 0xee, + 0xac, 0x59, 0x5e, 0x17, 0x36, 0x4a, 0x58, 0x8e, 0x04, 0x66, 0xcc, 0x57, 0xbe, 0x0b, 0x52, 0xb2, + 0x79, 0x84, 0x40, 0x9c, 0xda, 0x2e, 0x8f, 0x6d, 0x1d, 0xb3, 0x6f, 0x74, 0x0e, 0x96, 0x9e, 0xd8, + 0x93, 0x39, 0xcf, 0xce, 0x3a, 0xe6, 0x84, 0xa2, 0x42, 0x99, 0xef, 0x17, 0x5d, 0x86, 0x7a, 0xb2, + 0x8a, 0xe5, 0x06, 0x4c, 0xad, 0x84, 0x6b, 0x09, 0x6f, 0x10, 0xa4, 0x2e, 0xa8, 0x5f, 0x21, 0x76, + 0xf1, 0x9b, 0x22, 0x2c, 0xe7, 0x73, 0x14, 0xdd, 0x04, 0x31, 0x3c, 0x9c, 0xc5, 0x67, 0xfb, 0xfe, + 0x9b, 0x72, 0x39, 0x22, 0xcd, 0xc3, 0x19, 0xc1, 0xcc, 0x00, 0x7d, 0x0b, 0x90, 0xcb, 0x78, 0xd6, + 0x9e, 0xed, 0x3a, 0x93, 0x43, 0x8b, 0x6d, 0x83, 0x42, 0x91, 0xb0, 0xcc, 0x25, 0x77, 0x99, 0x40, + 0xa7, 0x5b, 0x42, 0x20, 0x1e, 0x90, 0xc9, 0xac, 0x29, 0x32, 0x39, 0xfb, 0xa6, 0xbc, 0xf9, 0xd4, + 0x09, 0x9b, 0x4b, 0x9c, 0x47, 0xbf, 0x95, 0x43, 0x80, 0x74, 0x25, 0x54, 0x83, 0xca, 0x48, 0x7f, + 0xa0, 0x0f, 0x1f, 0xe9, 0x72, 0x81, 0x12, 0x9d, 0xe1, 0x48, 0x37, 0x35, 0x2c, 0x0b, 0x48, 0x82, + 0xa5, 0x7b, 0xea, 0xe8, 0x9e, 0x26, 0x17, 0x51, 0x03, 0xa4, 0xad, 0x9e, 0x61, 0x0e, 0xef, 0x61, + 0x75, 0x20, 0x97, 0x10, 0x82, 0x65, 0x26, 0x49, 0x79, 0x22, 0x35, 0x35, 0x46, 0x83, 0x81, 0x8a, + 0x1f, 0xcb, 0x4b, 0x34, 0xf3, 0x7b, 0xfa, 0xdd, 0xa1, 0x5c, 0x46, 0x75, 0xa8, 0x1a, 0xa6, 0x6a, + 0x6a, 0x86, 0x66, 0xca, 0x15, 0xe5, 0x01, 0x94, 0xf9, 0xd2, 0x6f, 0x21, 0x6b, 0x95, 0x9f, 0x0b, + 0x50, 0x8d, 0x33, 0xed, 0x6d, 0xdc, 0x82, 0x5c, 0x4a, 0xc4, 0xe7, 0x79, 0x2c, 0x11, 0x4a, 0xc7, + 0x12, 0x41, 0x79, 0xbd, 0x04, 0x52, 0x92, 0xb9, 0xe8, 0x12, 0x48, 0xbc, 0x82, 0x38, 0xd3, 0x90, + 0x1d, 0xb9, 0xb8, 0x55, 0xc0, 0x55, 0xc6, 0xea, 0x4d, 0x43, 0x74, 0x19, 0x6a, 0x5c, 0xbc, 0x37, + 0xf1, 0xec, 0x90, 0xaf, 0xb5, 0x55, 0xc0, 0xc0, 0x98, 0x77, 0x29, 0x0f, 0xc9, 0x50, 0x0a, 0xe6, + 0x2e, 0x5b, 0x49, 0xc0, 0xf4, 0x13, 0x5d, 0x80, 0x72, 0x30, 0x3e, 0x20, 0xae, 0xcd, 0x0e, 0xf7, + 0x0c, 0x8e, 0x28, 0xf4, 0x7f, 0xb0, 0xfc, 0x13, 0xe2, 0x7b, 0x56, 0x78, 0xe0, 0x93, 0xe0, 0xc0, + 0x9b, 0xec, 0xb2, 0x83, 0x16, 0x70, 0x83, 0x72, 0xcd, 0x98, 0x89, 0x3e, 0x88, 0xd4, 0x52, 0x5c, + 0x65, 0x86, 0x4b, 0xc0, 0x75, 0xca, 0xef, 0xc4, 0xd8, 0xae, 0x80, 0x9c, 0xd1, 0xe3, 0x00, 0x2b, + 0x0c, 0xa0, 0x80, 0x97, 0x13, 0x4d, 0x0e, 0x52, 0x85, 0xe5, 0x29, 0xd9, 0xb7, 0x43, 0xe7, 0x09, + 0xb1, 0x82, 0x99, 0x3d, 0x0d, 0x9a, 0xd5, 0xc5, 0x57, 0xa3, 0x3d, 0x1f, 0x7f, 0x42, 0x42, 0x63, + 0x66, 0x4f, 0xa3, 0xeb, 0xdc, 0x88, 0x2d, 0x28, 0x2f, 0x40, 0xff, 0x0f, 0x2b, 0x89, 0x8b, 0x5d, + 0x32, 0x09, 0xed, 0xa0, 0x29, 0xad, 0x97, 0x36, 0x10, 0x4e, 0x3c, 0x77, 0x19, 0x37, 0xa7, 0xc8, + 0xb0, 0x05, 0x4d, 0x58, 0x2f, 0x6d, 0x08, 0xa9, 0x22, 0x03, 0x46, 0x6b, 0xe1, 0xf2, 0xcc, 0x0b, + 0x9c, 0x0c, 0xa8, 0xda, 0x97, 0x83, 0x8a, 0x2d, 0x12, 0x50, 0x89, 0x8b, 0x08, 0x54, 0x9d, 0x83, + 0x8a, 0xd9, 0x29, 0xa8, 0x44, 0x31, 0x02, 0xd5, 0xe0, 0xa0, 0x62, 0x76, 0x04, 0xea, 0x0e, 0x80, + 0x4f, 0x02, 0x12, 0x5a, 0x07, 0x34, 0xf2, 0xcb, 0xac, 0x08, 0x5c, 0x3a, 0xa1, 0xe6, 0x6d, 0x62, + 0xaa, 0xb5, 0xe5, 0x4c, 0x43, 0x2c, 0xf9, 0xf1, 0x27, 0x7a, 0x0f, 0xa4, 0xb4, 0xdc, 0xad, 0xb0, + 0xe4, 0x4b, 0x19, 0xe8, 0x7d, 0x68, 0x8c, 0xe7, 0x41, 0xe8, 0xb9, 0x16, 0xcb, 0xd6, 0xa0, 0x29, + 0x33, 0x08, 0x75, 0xce, 0x7c, 0xc8, 0x78, 0xca, 0x6d, 0x90, 0x12, 0xd7, 0xf9, 0xfb, 0x5e, 0x81, + 0xd2, 0x63, 0xcd, 0x90, 0x05, 0x54, 0x86, 0xa2, 0x3e, 0x94, 0x8b, 0xe9, 0x9d, 0x2f, 0xad, 0x8a, + 0xbf, 0xf8, 0x7d, 0x4b, 0x68, 0x57, 0x60, 0x89, 0x6d, 0xae, 0x5d, 0x07, 0x48, 0x73, 0x43, 0xf9, + 0xa7, 0x08, 0xcb, 0x2c, 0x0f, 0xd2, 0xbc, 0x0f, 0x00, 0x31, 0x19, 0xf1, 0xad, 0x85, 0xed, 0x36, + 0xda, 0xda, 0xbf, 0x5e, 0xac, 0xa9, 0x99, 0x16, 0x65, 0xe6, 0x7b, 0x2e, 0x09, 0x0f, 0xc8, 0x3c, + 0xc8, 0x7e, 0xba, 0xde, 0x2e, 0x99, 0x5c, 0x4d, 0x4a, 0xfe, 0x66, 0x87, 0xbb, 0x4b, 0xc3, 0x22, + 0x8f, 0x17, 0x38, 0xff, 0xed, 0xc5, 0xb8, 0x94, 0xdd, 0x14, 0x4f, 0x75, 0x2c, 0x25, 0x89, 0x4e, + 0x2b, 0x02, 0x97, 0x44, 0x15, 0x81, 0x11, 0x27, 0x5c, 0xcf, 0xb7, 0x90, 0x76, 0x6f, 0xe1, 0x3a, + 0x7d, 0x03, 0xe4, 0x04, 0xc5, 0x0e, 0xd3, 0x8d, 0x33, 0x32, 0x49, 0x54, 0xee, 0x82, 0xa9, 0x26, + 0xab, 0xc5, 0xaa, 0xfc, 0x46, 0x25, 0x17, 0x2d, 0x56, 0x3d, 0x4d, 0x86, 0xdd, 0x17, 0xab, 0x82, + 0x5c, 0xbc, 0x2f, 0x56, 0xcb, 0x72, 0xe5, 0xbe, 0x58, 0x95, 0x64, 0xb8, 0x2f, 0x56, 0xeb, 0x72, + 0xe3, 0xbe, 0x58, 0x5d, 0x91, 0x65, 0x9c, 0xd6, 0x43, 0xbc, 0x50, 0x87, 0xf0, 0x62, 0x01, 0xc0, + 0x8b, 0x97, 0x2f, 0x93, 0xec, 0xca, 0x1d, 0x80, 0x34, 0x06, 0xf4, 0xe8, 0xbd, 0xbd, 0xbd, 0x80, + 0xf0, 0x22, 0x7b, 0x06, 0x47, 0x14, 0xe5, 0x4f, 0xc8, 0x74, 0x3f, 0x3c, 0x60, 0xa7, 0xd6, 0xc0, + 0x11, 0xa5, 0xcc, 0x01, 0xe5, 0x33, 0x96, 0xf5, 0x06, 0xa7, 0x78, 0xe7, 0xef, 0x80, 0x94, 0xe4, + 0x24, 0x5b, 0x2b, 0xd7, 0x8f, 0xe6, 0x7d, 0x46, 0xfd, 0x68, 0x6a, 0xa0, 0x4c, 0x61, 0x85, 0xb7, + 0x14, 0xe9, 0x4d, 0x49, 0xd2, 0x4a, 0x38, 0x21, 0xad, 0x8a, 0x69, 0x5a, 0x5d, 0x87, 0x4a, 0x7c, + 0x38, 0xbc, 0xc5, 0xba, 0x78, 0x52, 0xa7, 0xc4, 0x34, 0x70, 0xac, 0xa9, 0x04, 0xb0, 0xb2, 0x20, + 0x43, 0x2d, 0x80, 0x1d, 0x6f, 0x3e, 0xdd, 0xb5, 0xa3, 0xe6, 0x5e, 0xd8, 0x58, 0xc2, 0x19, 0x0e, + 0xc5, 0x33, 0xf1, 0x3e, 0x25, 0x7e, 0x9c, 0xe6, 0x8c, 0xa0, 0xdc, 0xf9, 0x6c, 0x46, 0xfc, 0x28, + 0xd1, 0x39, 0x91, 0x62, 0x17, 0x33, 0xd8, 0x95, 0x09, 0x9c, 0x5d, 0xd8, 0x24, 0x0b, 0x6e, 0xae, + 0x76, 0x15, 0x17, 0x6b, 0xd7, 0xcd, 0xe3, 0x71, 0xbd, 0xb8, 0xd8, 0x77, 0x26, 0xfe, 0xb2, 0x21, + 0xfd, 0x5c, 0x84, 0xc6, 0xc7, 0x73, 0xe2, 0x1f, 0xc6, 0xed, 0x34, 0xba, 0x01, 0xe5, 0x20, 0xb4, + 0xc3, 0x79, 0x10, 0xf5, 0x58, 0xad, 0xd4, 0x4f, 0x4e, 0x71, 0xd3, 0x60, 0x5a, 0x38, 0xd2, 0x46, + 0x3f, 0x00, 0x20, 0xb4, 0xbf, 0xb6, 0x58, 0x7f, 0x76, 0xec, 0x47, 0x25, 0x6f, 0xcb, 0x3a, 0x71, + 0xd6, 0x9d, 0x49, 0x24, 0xfe, 0xa4, 0xf1, 0x60, 0x04, 0x8b, 0x92, 0x84, 0x39, 0x81, 0x36, 0x29, + 0x1e, 0xdf, 0x99, 0xee, 0xb3, 0x30, 0xe5, 0x6e, 0xb1, 0xc1, 0xf8, 0x5d, 0x3b, 0xb4, 0xb7, 0x0a, + 0x38, 0xd2, 0xa2, 0xfa, 0x4f, 0xc8, 0x38, 0xf4, 0x7c, 0x56, 0xa6, 0x72, 0xfa, 0x0f, 0x19, 0x3f, + 0xd6, 0xe7, 0x5a, 0xcc, 0xff, 0xd8, 0x9e, 0xd8, 0x3e, 0x7b, 0xc8, 0xf3, 0xfe, 0x19, 0x3f, 0xf1, + 0xcf, 0x28, 0xaa, 0xef, 0xda, 0xa1, 0xef, 0x3c, 0x65, 0x35, 0x2e, 0xa7, 0x3f, 0x60, 0xfc, 0x58, + 0x9f, 0x6b, 0xa1, 0x55, 0xa8, 0x7e, 0x6a, 0xfb, 0x53, 0x67, 0xba, 0xcf, 0xeb, 0x90, 0x84, 0x13, + 0x9a, 0xee, 0xd8, 0x99, 0xee, 0x79, 0xfc, 0xad, 0x96, 0x30, 0x27, 0x94, 0x0f, 0xa0, 0xcc, 0x63, + 0x4b, 0x9f, 0x10, 0x0d, 0xe3, 0x21, 0xe6, 0xed, 0xa4, 0x31, 0xea, 0x74, 0x34, 0xc3, 0x90, 0x05, + 0xfe, 0x9e, 0x28, 0xbf, 0x16, 0x40, 0x4a, 0x02, 0x49, 0xfb, 0x44, 0x7d, 0xa8, 0x6b, 0x5c, 0xd5, + 0xec, 0x0d, 0xb4, 0xe1, 0xc8, 0x94, 0x05, 0xda, 0x34, 0x76, 0x54, 0xbd, 0xa3, 0xf5, 0xb5, 0x2e, + 0x6f, 0x3e, 0xb5, 0x1f, 0x6a, 0x9d, 0x91, 0xd9, 0x1b, 0xea, 0x72, 0x89, 0x0a, 0xdb, 0x6a, 0xd7, + 0xea, 0xaa, 0xa6, 0x2a, 0x8b, 0x94, 0xea, 0xd1, 0x7e, 0x55, 0x57, 0xfb, 0xf2, 0x12, 0x5a, 0x81, + 0xda, 0x48, 0x57, 0x1f, 0xaa, 0xbd, 0xbe, 0xda, 0xee, 0x6b, 0x72, 0x99, 0xda, 0xea, 0x43, 0xd3, + 0xba, 0x3b, 0x1c, 0xe9, 0x5d, 0xb9, 0x42, 0x1b, 0x57, 0x4a, 0xaa, 0x9d, 0x8e, 0xb6, 0x6d, 0x32, + 0x95, 0x6a, 0xf4, 0xce, 0x95, 0x41, 0xa4, 0x3d, 0xb8, 0xa2, 0x01, 0xa4, 0x27, 0x94, 0x6f, 0xf1, + 0xa5, 0x37, 0xb5, 0x84, 0xc7, 0x6b, 0x86, 0xf2, 0x33, 0x01, 0x20, 0x3d, 0x39, 0x74, 0x23, 0xfd, + 0xc1, 0xe2, 0xed, 0xe9, 0x85, 0xc5, 0x03, 0x3e, 0xf9, 0x37, 0xeb, 0xfb, 0xb9, 0xdf, 0xa5, 0xe2, + 0x62, 0x11, 0xe0, 0xa6, 0xff, 0xe1, 0xa7, 0x49, 0xb1, 0xa0, 0x9e, 0xf5, 0x4f, 0x8b, 0x23, 0xff, + 0x6f, 0x60, 0x38, 0x24, 0x1c, 0x51, 0x5f, 0xbf, 0xf7, 0xfd, 0xa5, 0x00, 0x2b, 0x0b, 0x30, 0xde, + 0xb8, 0x48, 0xae, 0x90, 0x16, 0x4f, 0x51, 0x48, 0x0b, 0x99, 0x5b, 0x7f, 0x1a, 0x30, 0xf4, 0xf0, + 0x92, 0xf4, 0x3f, 0xf9, 0xff, 0xec, 0x34, 0x87, 0xd7, 0x06, 0x48, 0x6f, 0x05, 0xfa, 0x10, 0xca, + 0xb9, 0xb1, 0xc8, 0x85, 0xc5, 0xbb, 0x13, 0x0d, 0x46, 0x38, 0xe0, 0x48, 0x57, 0xf9, 0xad, 0x00, + 0xf5, 0xac, 0xf8, 0x8d, 0x41, 0xf9, 0xea, 0xff, 0xde, 0xed, 0x5c, 0x52, 0xf0, 0x97, 0xe1, 0xbd, + 0x37, 0xc5, 0x91, 0xfd, 0xf7, 0x1c, 0xcf, 0x8b, 0xbf, 0x0a, 0xb0, 0x92, 0x1d, 0xc3, 0xe0, 0x47, + 0xd7, 0x50, 0x13, 0x2a, 0xc1, 0xa1, 0xbb, 0xe3, 0x4d, 0xf8, 0x8f, 0xb9, 0x84, 0x63, 0x12, 0x7d, + 0x94, 0x9b, 0x0e, 0x2d, 0xb1, 0x15, 0xdf, 0x39, 0x69, 0x3a, 0x84, 0x1f, 0x5d, 0x8b, 0x17, 0xcb, + 0x8c, 0x82, 0xfe, 0x27, 0x33, 0x99, 0x3f, 0x15, 0xa1, 0x91, 0x03, 0x86, 0xd6, 0xa0, 0xc6, 0xff, + 0xf4, 0x2c, 0x9f, 0xec, 0xf1, 0xd3, 0x6c, 0x60, 0xe0, 0x2c, 0x4c, 0xf6, 0xbe, 0xce, 0x18, 0xe4, + 0xd6, 0x09, 0x47, 0x71, 0xca, 0x71, 0xc6, 0xad, 0xec, 0x04, 0x85, 0x0f, 0x42, 0xce, 0x1f, 0x9f, + 0xa0, 0xa4, 0x01, 0xcd, 0x0c, 0x51, 0x6e, 0x66, 0xe6, 0x63, 0xfc, 0xbd, 0x38, 0x9f, 0x9b, 0x29, + 0x30, 0x49, 0x6a, 0x99, 0x0e, 0xc7, 0xbe, 0xd2, 0x08, 0x65, 0x07, 0x6a, 0x19, 0x14, 0x5f, 0x1e, + 0xbd, 0x93, 0x4b, 0x48, 0xae, 0x05, 0x28, 0x2d, 0xb4, 0x00, 0xca, 0xe7, 0x45, 0xa8, 0x65, 0x00, + 0xa3, 0x0f, 0x73, 0x93, 0x92, 0xf5, 0x13, 0x77, 0x75, 0x7c, 0x4c, 0x72, 0x11, 0xaa, 0x07, 0x64, + 0x32, 0xa3, 0xc0, 0xd8, 0x12, 0x0d, 0x5c, 0xa1, 0x34, 0x26, 0x7b, 0x54, 0x34, 0x9f, 0x3a, 0x21, + 0x13, 0x89, 0x5c, 0x44, 0x69, 0x4c, 0xf6, 0x94, 0xbf, 0x08, 0xb9, 0x39, 0xc8, 0xbb, 0xf0, 0xce, + 0x40, 0x33, 0x71, 0xaf, 0x63, 0x99, 0x8f, 0xb7, 0x35, 0x6b, 0xa4, 0x1b, 0xdb, 0x5a, 0xa7, 0x77, + 0xb7, 0xa7, 0x75, 0xe5, 0x02, 0x7a, 0x07, 0xce, 0x66, 0x85, 0xe9, 0x8c, 0xe4, 0x3c, 0x9c, 0xc9, + 0x0a, 0xe2, 0x79, 0xc9, 0x45, 0x38, 0x9f, 0x65, 0x67, 0x67, 0x27, 0x2d, 0x58, 0x3d, 0x66, 0x91, + 0x9d, 0xa3, 0x2c, 0x2c, 0x95, 0xce, 0x54, 0xce, 0x81, 0x9c, 0x15, 0x44, 0xf3, 0x95, 0x26, 0x9c, + 0xcb, 0xa9, 0x27, 0xb3, 0x96, 0x2b, 0x7f, 0x2c, 0x02, 0xa4, 0xc3, 0x43, 0x74, 0x06, 0x1a, 0xd1, + 0x7f, 0x9f, 0xd5, 0x51, 0x47, 0x06, 0x7d, 0x73, 0x57, 0xe1, 0x02, 0xd6, 0xb6, 0xfb, 0xbd, 0x8e, + 0x6a, 0x58, 0xdd, 0x5e, 0xd7, 0xa2, 0x4f, 0xe3, 0x40, 0x35, 0x3b, 0x5b, 0x7c, 0x63, 0xe6, 0x70, + 0x68, 0x0d, 0x54, 0xfd, 0xb1, 0xd5, 0xe9, 0x8f, 0x0c, 0x53, 0xc3, 0x86, 0x5c, 0xcc, 0x3d, 0xbe, + 0x25, 0xea, 0xa0, 0xa7, 0xdf, 0xd3, 0x0c, 0xfa, 0x32, 0x5b, 0x58, 0x35, 0x35, 0xab, 0xdf, 0x1b, + 0xf4, 0x4c, 0xad, 0x2b, 0x8b, 0x14, 0x18, 0xd6, 0x3e, 0x1e, 0x69, 0x86, 0x99, 0x97, 0x2c, 0xd1, + 0x47, 0xb8, 0xa7, 0x1b, 0x26, 0x7d, 0xe0, 0x39, 0x57, 0x2e, 0xd3, 0x5d, 0x1b, 0x1a, 0x7e, 0xd8, + 0xeb, 0xd0, 0xc8, 0xa7, 0x0f, 0x78, 0x85, 0xee, 0xda, 0x34, 0xba, 0xed, 0x1c, 0xb7, 0x4a, 0x61, + 0x50, 0x74, 0xed, 0x91, 0xf1, 0x58, 0x96, 0xe8, 0x52, 0x9d, 0x1e, 0xee, 0x8c, 0x7a, 0xa6, 0xd5, + 0xc6, 0x9a, 0xfa, 0x40, 0xc3, 0xd6, 0x70, 0x5b, 0xd3, 0x65, 0x40, 0x17, 0x00, 0x0d, 0x34, 0x73, + 0x6b, 0xc8, 0xf7, 0xa6, 0xf6, 0xfb, 0xc3, 0x47, 0x5a, 0x57, 0xae, 0x21, 0x19, 0xea, 0xa6, 0xa6, + 0xab, 0xba, 0x19, 0x01, 0xa8, 0xb7, 0x3f, 0x7a, 0xfe, 0xb2, 0x55, 0xf8, 0xec, 0x65, 0xab, 0xf0, + 0xfa, 0x65, 0x4b, 0xf8, 0xe9, 0x51, 0x4b, 0xf8, 0xc3, 0x51, 0x4b, 0x78, 0x76, 0xd4, 0x12, 0x9e, + 0x1f, 0xb5, 0x84, 0xbf, 0x1d, 0xb5, 0x84, 0x2f, 0x8e, 0x5a, 0x85, 0xd7, 0x47, 0x2d, 0xe1, 0x57, + 0xaf, 0x5a, 0x85, 0xe7, 0xaf, 0x5a, 0x85, 0xcf, 0x5e, 0xb5, 0x0a, 0x3f, 0xaa, 0xb0, 0x69, 0xfc, + 0x6c, 0x67, 0xa7, 0xcc, 0xe6, 0xea, 0xd7, 0xff, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x60, 0x8e, 0x69, + 0x12, 0x9f, 0x17, 0x00, 0x00, } func (x ErrorCause) String() string { @@ -2097,6 +2498,13 @@ func (x QueryResponse_ErrorType) String() string { } return strconv.Itoa(int(x)) } +func (x MetadataRW2_MetricType) String() string { + s, ok := MetadataRW2_MetricType_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} func (this *WriteRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -2239,6 +2647,9 @@ func (this *TimeSeries) Equal(that interface{}) bool { return false } } + if this.CreatedTimestamp != that1.CreatedTimestamp { + return false + } return true } func (this *LabelPair) Equal(that interface{}) bool { @@ -2492,6 +2903,14 @@ func (this *Histogram) Equal(that interface{}) bool { if this.Timestamp != that1.Timestamp { return false } + if len(this.CustomValues) != len(that1.CustomValues) { + return false + } + for i := range this.CustomValues { + if this.CustomValues[i] != that1.CustomValues[i] { + return false + } + } return true } func (this *Histogram_CountInt) Equal(that interface{}) bool { @@ -3204,34 +3623,201 @@ func (this *MatrixSeries) Equal(that interface{}) bool { } return true } -func (this *WriteRequest) GoString() string { - if this == nil { - return "nil" +func (this *WriteRequestRW2) Equal(that interface{}) bool { + if that == nil { + return this == nil } - s := make([]string, 0, 9) - s = append(s, "&mimirpb.WriteRequest{") - s = append(s, "Timeseries: "+fmt.Sprintf("%#v", this.Timeseries)+",\n") - s = append(s, "Source: "+fmt.Sprintf("%#v", this.Source)+",\n") - if this.Metadata != nil { - s = append(s, "Metadata: "+fmt.Sprintf("%#v", this.Metadata)+",\n") + + that1, ok := that.(*WriteRequestRW2) + if !ok { + that2, ok := that.(WriteRequestRW2) + if ok { + that1 = &that2 + } else { + return false + } } - s = append(s, "SkipLabelValidation: "+fmt.Sprintf("%#v", this.SkipLabelValidation)+",\n") - s = append(s, "SkipLabelCountValidation: "+fmt.Sprintf("%#v", this.SkipLabelCountValidation)+",\n") - s = append(s, "}") - return strings.Join(s, "") -} -func (this *WriteResponse) GoString() string { - if this == nil { - return "nil" + if that1 == nil { + return this == nil + } else if this == nil { + return false } - s := make([]string, 0, 4) - s = append(s, "&mimirpb.WriteResponse{") - s = append(s, "}") - return strings.Join(s, "") -} -func (this *ErrorDetails) GoString() string { - if this == nil { - return "nil" + if len(this.Symbols) != len(that1.Symbols) { + return false + } + for i := range this.Symbols { + if this.Symbols[i] != that1.Symbols[i] { + return false + } + } + if len(this.Timeseries) != len(that1.Timeseries) { + return false + } + for i := range this.Timeseries { + if !this.Timeseries[i].Equal(&that1.Timeseries[i]) { + return false + } + } + if this.SkipLabelValidation != that1.SkipLabelValidation { + return false + } + if this.SkipLabelCountValidation != that1.SkipLabelCountValidation { + return false + } + return true +} +func (this *TimeSeriesRW2) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*TimeSeriesRW2) + if !ok { + that2, ok := that.(TimeSeriesRW2) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.LabelsRefs) != len(that1.LabelsRefs) { + return false + } + for i := range this.LabelsRefs { + if this.LabelsRefs[i] != that1.LabelsRefs[i] { + return false + } + } + if len(this.Samples) != len(that1.Samples) { + return false + } + for i := range this.Samples { + if !this.Samples[i].Equal(&that1.Samples[i]) { + return false + } + } + if len(this.Histograms) != len(that1.Histograms) { + return false + } + for i := range this.Histograms { + if !this.Histograms[i].Equal(&that1.Histograms[i]) { + return false + } + } + if len(this.Exemplars) != len(that1.Exemplars) { + return false + } + for i := range this.Exemplars { + if !this.Exemplars[i].Equal(&that1.Exemplars[i]) { + return false + } + } + if !this.Metadata.Equal(&that1.Metadata) { + return false + } + if this.CreatedTimestamp != that1.CreatedTimestamp { + return false + } + return true +} +func (this *ExemplarRW2) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ExemplarRW2) + if !ok { + that2, ok := that.(ExemplarRW2) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.LabelsRefs) != len(that1.LabelsRefs) { + return false + } + for i := range this.LabelsRefs { + if this.LabelsRefs[i] != that1.LabelsRefs[i] { + return false + } + } + if this.Value != that1.Value { + return false + } + if this.Timestamp != that1.Timestamp { + return false + } + return true +} +func (this *MetadataRW2) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*MetadataRW2) + if !ok { + that2, ok := that.(MetadataRW2) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Type != that1.Type { + return false + } + if this.HelpRef != that1.HelpRef { + return false + } + if this.UnitRef != that1.UnitRef { + return false + } + return true +} +func (this *WriteRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 9) + s = append(s, "&mimirpb.WriteRequest{") + s = append(s, "Timeseries: "+fmt.Sprintf("%#v", this.Timeseries)+",\n") + s = append(s, "Source: "+fmt.Sprintf("%#v", this.Source)+",\n") + if this.Metadata != nil { + s = append(s, "Metadata: "+fmt.Sprintf("%#v", this.Metadata)+",\n") + } + s = append(s, "SkipLabelValidation: "+fmt.Sprintf("%#v", this.SkipLabelValidation)+",\n") + s = append(s, "SkipLabelCountValidation: "+fmt.Sprintf("%#v", this.SkipLabelCountValidation)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *WriteResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 4) + s = append(s, "&mimirpb.WriteResponse{") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ErrorDetails) GoString() string { + if this == nil { + return "nil" } s := make([]string, 0, 5) s = append(s, "&mimirpb.ErrorDetails{") @@ -3243,7 +3829,7 @@ func (this *TimeSeries) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 8) + s := make([]string, 0, 9) s = append(s, "&mimirpb.TimeSeries{") s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") if this.Samples != nil { @@ -3267,6 +3853,7 @@ func (this *TimeSeries) GoString() string { } s = append(s, "Histograms: "+fmt.Sprintf("%#v", vs)+",\n") } + s = append(s, "CreatedTimestamp: "+fmt.Sprintf("%#v", this.CreatedTimestamp)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -3331,7 +3918,7 @@ func (this *Histogram) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 19) + s := make([]string, 0, 20) s = append(s, "&mimirpb.Histogram{") if this.Count != nil { s = append(s, "Count: "+fmt.Sprintf("%#v", this.Count)+",\n") @@ -3362,6 +3949,7 @@ func (this *Histogram) GoString() string { s = append(s, "PositiveCounts: "+fmt.Sprintf("%#v", this.PositiveCounts)+",\n") s = append(s, "ResetHint: "+fmt.Sprintf("%#v", this.ResetHint)+",\n") s = append(s, "Timestamp: "+fmt.Sprintf("%#v", this.Timestamp)+",\n") + s = append(s, "CustomValues: "+fmt.Sprintf("%#v", this.CustomValues)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -3651,6 +4239,82 @@ func (this *MatrixSeries) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *WriteRequestRW2) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 8) + s = append(s, "&mimirpb.WriteRequestRW2{") + s = append(s, "Symbols: "+fmt.Sprintf("%#v", this.Symbols)+",\n") + if this.Timeseries != nil { + vs := make([]*TimeSeriesRW2, len(this.Timeseries)) + for i := range vs { + vs[i] = &this.Timeseries[i] + } + s = append(s, "Timeseries: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "SkipLabelValidation: "+fmt.Sprintf("%#v", this.SkipLabelValidation)+",\n") + s = append(s, "SkipLabelCountValidation: "+fmt.Sprintf("%#v", this.SkipLabelCountValidation)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *TimeSeriesRW2) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 10) + s = append(s, "&mimirpb.TimeSeriesRW2{") + s = append(s, "LabelsRefs: "+fmt.Sprintf("%#v", this.LabelsRefs)+",\n") + if this.Samples != nil { + vs := make([]*Sample, len(this.Samples)) + for i := range vs { + vs[i] = &this.Samples[i] + } + s = append(s, "Samples: "+fmt.Sprintf("%#v", vs)+",\n") + } + if this.Histograms != nil { + vs := make([]*Histogram, len(this.Histograms)) + for i := range vs { + vs[i] = &this.Histograms[i] + } + s = append(s, "Histograms: "+fmt.Sprintf("%#v", vs)+",\n") + } + if this.Exemplars != nil { + vs := make([]*ExemplarRW2, len(this.Exemplars)) + for i := range vs { + vs[i] = &this.Exemplars[i] + } + s = append(s, "Exemplars: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "Metadata: "+strings.Replace(this.Metadata.GoString(), `&`, ``, 1)+",\n") + s = append(s, "CreatedTimestamp: "+fmt.Sprintf("%#v", this.CreatedTimestamp)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ExemplarRW2) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&mimirpb.ExemplarRW2{") + s = append(s, "LabelsRefs: "+fmt.Sprintf("%#v", this.LabelsRefs)+",\n") + s = append(s, "Value: "+fmt.Sprintf("%#v", this.Value)+",\n") + s = append(s, "Timestamp: "+fmt.Sprintf("%#v", this.Timestamp)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *MetadataRW2) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&mimirpb.MetadataRW2{") + s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n") + s = append(s, "HelpRef: "+fmt.Sprintf("%#v", this.HelpRef)+",\n") + s = append(s, "UnitRef: "+fmt.Sprintf("%#v", this.UnitRef)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} func valueToGoStringMimir(v interface{}, typ string) string { rv := reflect.ValueOf(v) if rv.IsNil() { @@ -3810,6 +4474,11 @@ func (m *TimeSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.CreatedTimestamp != 0 { + i = encodeVarintMimir(dAtA, i, uint64(m.CreatedTimestamp)) + i-- + dAtA[i] = 0x30 + } if len(m.Histograms) > 0 { for iNdEx := len(m.Histograms) - 1; iNdEx >= 0; iNdEx-- { { @@ -4094,6 +4763,18 @@ func (m *Histogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.CustomValues) > 0 { + for iNdEx := len(m.CustomValues) - 1; iNdEx >= 0; iNdEx-- { + f1 := math.Float64bits(float64(m.CustomValues[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f1)) + } + i = encodeVarintMimir(dAtA, i, uint64(len(m.CustomValues)*8)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } if m.Timestamp != 0 { i = encodeVarintMimir(dAtA, i, uint64(m.Timestamp)) i-- @@ -4106,30 +4787,30 @@ func (m *Histogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { } if len(m.PositiveCounts) > 0 { for iNdEx := len(m.PositiveCounts) - 1; iNdEx >= 0; iNdEx-- { - f1 := math.Float64bits(float64(m.PositiveCounts[iNdEx])) + f2 := math.Float64bits(float64(m.PositiveCounts[iNdEx])) i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f1)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f2)) } i = encodeVarintMimir(dAtA, i, uint64(len(m.PositiveCounts)*8)) i-- dAtA[i] = 0x6a } if len(m.PositiveDeltas) > 0 { - var j2 int - dAtA4 := make([]byte, len(m.PositiveDeltas)*10) + var j3 int + dAtA5 := make([]byte, len(m.PositiveDeltas)*10) for _, num := range m.PositiveDeltas { - x3 := (uint64(num) << 1) ^ uint64((num >> 63)) - for x3 >= 1<<7 { - dAtA4[j2] = uint8(uint64(x3)&0x7f | 0x80) - j2++ - x3 >>= 7 - } - dAtA4[j2] = uint8(x3) - j2++ - } - i -= j2 - copy(dAtA[i:], dAtA4[:j2]) - i = encodeVarintMimir(dAtA, i, uint64(j2)) + x4 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x4 >= 1<<7 { + dAtA5[j3] = uint8(uint64(x4)&0x7f | 0x80) + j3++ + x4 >>= 7 + } + dAtA5[j3] = uint8(x4) + j3++ + } + i -= j3 + copy(dAtA[i:], dAtA5[:j3]) + i = encodeVarintMimir(dAtA, i, uint64(j3)) i-- dAtA[i] = 0x62 } @@ -4149,30 +4830,30 @@ func (m *Histogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { } if len(m.NegativeCounts) > 0 { for iNdEx := len(m.NegativeCounts) - 1; iNdEx >= 0; iNdEx-- { - f5 := math.Float64bits(float64(m.NegativeCounts[iNdEx])) + f6 := math.Float64bits(float64(m.NegativeCounts[iNdEx])) i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f5)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f6)) } i = encodeVarintMimir(dAtA, i, uint64(len(m.NegativeCounts)*8)) i-- dAtA[i] = 0x52 } if len(m.NegativeDeltas) > 0 { - var j6 int - dAtA8 := make([]byte, len(m.NegativeDeltas)*10) + var j7 int + dAtA9 := make([]byte, len(m.NegativeDeltas)*10) for _, num := range m.NegativeDeltas { - x7 := (uint64(num) << 1) ^ uint64((num >> 63)) - for x7 >= 1<<7 { - dAtA8[j6] = uint8(uint64(x7)&0x7f | 0x80) - j6++ - x7 >>= 7 - } - dAtA8[j6] = uint8(x7) - j6++ - } - i -= j6 - copy(dAtA[i:], dAtA8[:j6]) - i = encodeVarintMimir(dAtA, i, uint64(j6)) + x8 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x8 >= 1<<7 { + dAtA9[j7] = uint8(uint64(x8)&0x7f | 0x80) + j7++ + x8 >>= 7 + } + dAtA9[j7] = uint8(x8) + j7++ + } + i -= j7 + copy(dAtA[i:], dAtA9[:j7]) + i = encodeVarintMimir(dAtA, i, uint64(j7)) i-- dAtA[i] = 0x4a } @@ -4296,9 +4977,9 @@ func (m *FloatHistogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = l if len(m.CustomValues) > 0 { for iNdEx := len(m.CustomValues) - 1; iNdEx >= 0; iNdEx-- { - f9 := math.Float64bits(float64(m.CustomValues[iNdEx])) + f10 := math.Float64bits(float64(m.CustomValues[iNdEx])) i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f9)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f10)) } i = encodeVarintMimir(dAtA, i, uint64(len(m.CustomValues)*8)) i-- @@ -4313,9 +4994,9 @@ func (m *FloatHistogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { } if len(m.PositiveBuckets) > 0 { for iNdEx := len(m.PositiveBuckets) - 1; iNdEx >= 0; iNdEx-- { - f10 := math.Float64bits(float64(m.PositiveBuckets[iNdEx])) + f11 := math.Float64bits(float64(m.PositiveBuckets[iNdEx])) i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f10)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f11)) } i = encodeVarintMimir(dAtA, i, uint64(len(m.PositiveBuckets)*8)) i-- @@ -4337,9 +5018,9 @@ func (m *FloatHistogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { } if len(m.NegativeBuckets) > 0 { for iNdEx := len(m.NegativeBuckets) - 1; iNdEx >= 0; iNdEx-- { - f11 := math.Float64bits(float64(m.NegativeBuckets[iNdEx])) + f12 := math.Float64bits(float64(m.NegativeBuckets[iNdEx])) i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f11)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f12)) } i = encodeVarintMimir(dAtA, i, uint64(len(m.NegativeBuckets)*8)) i-- @@ -5053,24 +5734,282 @@ func (m *MatrixSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func encodeVarintMimir(dAtA []byte, offset int, v uint64) int { - offset -= sovMimir(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *WriteRequestRW2) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err } - dAtA[offset] = uint8(v) - return base + return dAtA[:n], nil } -func (m *WriteRequest) Size() (n int) { - if m == nil { - return 0 - } + +func (m *WriteRequestRW2) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *WriteRequestRW2) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if len(m.Timeseries) > 0 { + if m.SkipLabelCountValidation { + i-- + if m.SkipLabelCountValidation { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x3e + i-- + dAtA[i] = 0xc8 + } + if m.SkipLabelValidation { + i-- + if m.SkipLabelValidation { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x3e + i-- + dAtA[i] = 0xc0 + } + 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 = encodeVarintMimir(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 = encodeVarintMimir(dAtA, i, uint64(len(m.Symbols[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + return len(dAtA) - i, nil +} + +func (m *TimeSeriesRW2) 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 *TimeSeriesRW2) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TimeSeriesRW2) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.CreatedTimestamp != 0 { + i = encodeVarintMimir(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 = encodeVarintMimir(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 = encodeVarintMimir(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 = encodeVarintMimir(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 = encodeVarintMimir(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.LabelsRefs) > 0 { + dAtA22 := make([]byte, len(m.LabelsRefs)*10) + var j21 int + for _, num := range m.LabelsRefs { + for num >= 1<<7 { + dAtA22[j21] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j21++ + } + dAtA22[j21] = uint8(num) + j21++ + } + i -= j21 + copy(dAtA[i:], dAtA22[:j21]) + i = encodeVarintMimir(dAtA, i, uint64(j21)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ExemplarRW2) 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 *ExemplarRW2) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ExemplarRW2) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Timestamp != 0 { + i = encodeVarintMimir(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 { + dAtA24 := make([]byte, len(m.LabelsRefs)*10) + var j23 int + for _, num := range m.LabelsRefs { + for num >= 1<<7 { + dAtA24[j23] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j23++ + } + dAtA24[j23] = uint8(num) + j23++ + } + i -= j23 + copy(dAtA[i:], dAtA24[:j23]) + i = encodeVarintMimir(dAtA, i, uint64(j23)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *MetadataRW2) 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 *MetadataRW2) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MetadataRW2) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.UnitRef != 0 { + i = encodeVarintMimir(dAtA, i, uint64(m.UnitRef)) + i-- + dAtA[i] = 0x20 + } + if m.HelpRef != 0 { + i = encodeVarintMimir(dAtA, i, uint64(m.HelpRef)) + i-- + dAtA[i] = 0x18 + } + if m.Type != 0 { + i = encodeVarintMimir(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintMimir(dAtA []byte, offset int, v uint64) int { + offset -= sovMimir(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *WriteRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Timeseries) > 0 { for _, e := range m.Timeseries { l = e.Size() n += 1 + l + sovMimir(uint64(l)) @@ -5145,6 +6084,9 @@ func (m *TimeSeries) Size() (n int) { n += 1 + l + sovMimir(uint64(l)) } } + if m.CreatedTimestamp != 0 { + n += 1 + sovMimir(uint64(m.CreatedTimestamp)) + } return n } @@ -5299,6 +6241,9 @@ func (m *Histogram) Size() (n int) { if m.Timestamp != 0 { n += 1 + sovMimir(uint64(m.Timestamp)) } + if len(m.CustomValues) > 0 { + n += 2 + sovMimir(uint64(len(m.CustomValues)*8)) + len(m.CustomValues)*8 + } return n } @@ -5692,45 +6637,151 @@ func (m *MatrixSeries) Size() (n int) { return n } -func sovMimir(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozMimir(x uint64) (n int) { - return sovMimir(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} -func (this *WriteRequest) String() string { - if this == nil { - return "nil" +func (m *WriteRequestRW2) Size() (n int) { + if m == nil { + return 0 } - repeatedStringForMetadata := "[]*MetricMetadata{" - for _, f := range this.Metadata { - repeatedStringForMetadata += strings.Replace(f.String(), "MetricMetadata", "MetricMetadata", 1) + "," + var l int + _ = l + if len(m.Symbols) > 0 { + for _, s := range m.Symbols { + l = len(s) + n += 1 + l + sovMimir(uint64(l)) + } } - repeatedStringForMetadata += "}" - s := strings.Join([]string{`&WriteRequest{`, - `Timeseries:` + fmt.Sprintf("%v", this.Timeseries) + `,`, - `Source:` + fmt.Sprintf("%v", this.Source) + `,`, - `Metadata:` + repeatedStringForMetadata + `,`, - `SkipLabelValidation:` + fmt.Sprintf("%v", this.SkipLabelValidation) + `,`, - `SkipLabelCountValidation:` + fmt.Sprintf("%v", this.SkipLabelCountValidation) + `,`, - `}`, - }, "") - return s -} -func (this *WriteResponse) String() string { - if this == nil { - return "nil" + if len(m.Timeseries) > 0 { + for _, e := range m.Timeseries { + l = e.Size() + n += 1 + l + sovMimir(uint64(l)) + } } - s := strings.Join([]string{`&WriteResponse{`, - `}`, - }, "") - return s -} -func (this *ErrorDetails) String() string { - if this == nil { - return "nil" + if m.SkipLabelValidation { + n += 3 } - s := strings.Join([]string{`&ErrorDetails{`, + if m.SkipLabelCountValidation { + n += 3 + } + return n +} + +func (m *TimeSeriesRW2) 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 += sovMimir(uint64(e)) + } + n += 1 + sovMimir(uint64(l)) + l + } + if len(m.Samples) > 0 { + for _, e := range m.Samples { + l = e.Size() + n += 1 + l + sovMimir(uint64(l)) + } + } + if len(m.Histograms) > 0 { + for _, e := range m.Histograms { + l = e.Size() + n += 1 + l + sovMimir(uint64(l)) + } + } + if len(m.Exemplars) > 0 { + for _, e := range m.Exemplars { + l = e.Size() + n += 1 + l + sovMimir(uint64(l)) + } + } + l = m.Metadata.Size() + n += 1 + l + sovMimir(uint64(l)) + if m.CreatedTimestamp != 0 { + n += 1 + sovMimir(uint64(m.CreatedTimestamp)) + } + return n +} + +func (m *ExemplarRW2) 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 += sovMimir(uint64(e)) + } + n += 1 + sovMimir(uint64(l)) + l + } + if m.Value != 0 { + n += 9 + } + if m.Timestamp != 0 { + n += 1 + sovMimir(uint64(m.Timestamp)) + } + return n +} + +func (m *MetadataRW2) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Type != 0 { + n += 1 + sovMimir(uint64(m.Type)) + } + if m.HelpRef != 0 { + n += 1 + sovMimir(uint64(m.HelpRef)) + } + if m.UnitRef != 0 { + n += 1 + sovMimir(uint64(m.UnitRef)) + } + return n +} + +func sovMimir(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozMimir(x uint64) (n int) { + return sovMimir(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *WriteRequest) String() string { + if this == nil { + return "nil" + } + repeatedStringForMetadata := "[]*MetricMetadata{" + for _, f := range this.Metadata { + repeatedStringForMetadata += strings.Replace(f.String(), "MetricMetadata", "MetricMetadata", 1) + "," + } + repeatedStringForMetadata += "}" + s := strings.Join([]string{`&WriteRequest{`, + `Timeseries:` + fmt.Sprintf("%v", this.Timeseries) + `,`, + `Source:` + fmt.Sprintf("%v", this.Source) + `,`, + `Metadata:` + repeatedStringForMetadata + `,`, + `SkipLabelValidation:` + fmt.Sprintf("%v", this.SkipLabelValidation) + `,`, + `SkipLabelCountValidation:` + fmt.Sprintf("%v", this.SkipLabelCountValidation) + `,`, + `}`, + }, "") + return s +} +func (this *WriteResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&WriteResponse{`, + `}`, + }, "") + return s +} +func (this *ErrorDetails) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ErrorDetails{`, `Cause:` + fmt.Sprintf("%v", this.Cause) + `,`, `}`, }, "") @@ -5760,6 +6811,7 @@ func (this *TimeSeries) String() string { `Samples:` + repeatedStringForSamples + `,`, `Exemplars:` + repeatedStringForExemplars + `,`, `Histograms:` + repeatedStringForHistograms + `,`, + `CreatedTimestamp:` + fmt.Sprintf("%v", this.CreatedTimestamp) + `,`, `}`, }, "") return s @@ -5849,6 +6901,7 @@ func (this *Histogram) String() string { `PositiveCounts:` + fmt.Sprintf("%v", this.PositiveCounts) + `,`, `ResetHint:` + fmt.Sprintf("%v", this.ResetHint) + `,`, `Timestamp:` + fmt.Sprintf("%v", this.Timestamp) + `,`, + `CustomValues:` + fmt.Sprintf("%v", this.CustomValues) + `,`, `}`, }, "") return s @@ -6145,6 +7198,78 @@ func (this *MatrixSeries) String() string { }, "") return s } +func (this *WriteRequestRW2) String() string { + if this == nil { + return "nil" + } + repeatedStringForTimeseries := "[]TimeSeriesRW2{" + for _, f := range this.Timeseries { + repeatedStringForTimeseries += strings.Replace(strings.Replace(f.String(), "TimeSeriesRW2", "TimeSeriesRW2", 1), `&`, ``, 1) + "," + } + repeatedStringForTimeseries += "}" + s := strings.Join([]string{`&WriteRequestRW2{`, + `Symbols:` + fmt.Sprintf("%v", this.Symbols) + `,`, + `Timeseries:` + repeatedStringForTimeseries + `,`, + `SkipLabelValidation:` + fmt.Sprintf("%v", this.SkipLabelValidation) + `,`, + `SkipLabelCountValidation:` + fmt.Sprintf("%v", this.SkipLabelCountValidation) + `,`, + `}`, + }, "") + return s +} +func (this *TimeSeriesRW2) String() string { + if this == nil { + return "nil" + } + repeatedStringForSamples := "[]Sample{" + for _, f := range this.Samples { + repeatedStringForSamples += strings.Replace(strings.Replace(f.String(), "Sample", "Sample", 1), `&`, ``, 1) + "," + } + repeatedStringForSamples += "}" + repeatedStringForHistograms := "[]Histogram{" + for _, f := range this.Histograms { + repeatedStringForHistograms += strings.Replace(strings.Replace(f.String(), "Histogram", "Histogram", 1), `&`, ``, 1) + "," + } + repeatedStringForHistograms += "}" + repeatedStringForExemplars := "[]ExemplarRW2{" + for _, f := range this.Exemplars { + repeatedStringForExemplars += strings.Replace(strings.Replace(f.String(), "ExemplarRW2", "ExemplarRW2", 1), `&`, ``, 1) + "," + } + repeatedStringForExemplars += "}" + s := strings.Join([]string{`&TimeSeriesRW2{`, + `LabelsRefs:` + fmt.Sprintf("%v", this.LabelsRefs) + `,`, + `Samples:` + repeatedStringForSamples + `,`, + `Histograms:` + repeatedStringForHistograms + `,`, + `Exemplars:` + repeatedStringForExemplars + `,`, + `Metadata:` + strings.Replace(strings.Replace(this.Metadata.String(), "MetadataRW2", "MetadataRW2", 1), `&`, ``, 1) + `,`, + `CreatedTimestamp:` + fmt.Sprintf("%v", this.CreatedTimestamp) + `,`, + `}`, + }, "") + return s +} +func (this *ExemplarRW2) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ExemplarRW2{`, + `LabelsRefs:` + fmt.Sprintf("%v", this.LabelsRefs) + `,`, + `Value:` + fmt.Sprintf("%v", this.Value) + `,`, + `Timestamp:` + fmt.Sprintf("%v", this.Timestamp) + `,`, + `}`, + }, "") + return s +} +func (this *MetadataRW2) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&MetadataRW2{`, + `Type:` + fmt.Sprintf("%v", this.Type) + `,`, + `HelpRef:` + fmt.Sprintf("%v", this.HelpRef) + `,`, + `UnitRef:` + fmt.Sprintf("%v", this.UnitRef) + `,`, + `}`, + }, "") + return s +} func valueToStringMimir(v interface{}) string { rv := reflect.ValueOf(v) if rv.IsNil() { @@ -6586,8 +7711,8 @@ func (m *TimeSeries) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if !m.SkipUnmarshalingExemplars { - m.Exemplars = append(m.Exemplars, Exemplar{}) - if err := m.Exemplars[len(m.Exemplars)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Exemplars = append(m.Exemplars, Exemplar{}) + if err := m.Exemplars[len(m.Exemplars)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } } @@ -6626,6 +7751,25 @@ func (m *TimeSeries) Unmarshal(dAtA []byte) error { 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 ErrIntOverflowMimir + } + 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 := skipMimir(dAtA[iNdEx:]) @@ -7730,6 +8874,60 @@ func (m *Histogram) Unmarshal(dAtA []byte) error { 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthMimir + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + 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 := skipMimir(dAtA[iNdEx:]) @@ -9771,6 +10969,717 @@ func (m *MatrixSeries) Unmarshal(dAtA []byte) error { } return nil } +func (m *WriteRequestRW2) 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 ErrIntOverflowMimir + } + 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: WriteRequestRW2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WriteRequestRW2: 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 ErrIntOverflowMimir + } + 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 ErrInvalidLengthMimir + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMimir + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Timeseries = append(m.Timeseries, TimeSeriesRW2{}) + if err := m.Timeseries[len(m.Timeseries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 1000: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SkipLabelValidation", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.SkipLabelValidation = bool(v != 0) + case 1001: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SkipLabelCountValidation", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.SkipLabelCountValidation = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipMimir(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *TimeSeriesRW2) 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 ErrIntOverflowMimir + } + 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: TimeSeriesRW2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TimeSeriesRW2: 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 ErrIntOverflowMimir + } + 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthMimir + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + 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 ErrIntOverflowMimir + } + 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMimir + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMimir + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMimir + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Exemplars = append(m.Exemplars, ExemplarRW2{}) + 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMimir + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + 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 ErrIntOverflowMimir + } + 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 := skipMimir(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExemplarRW2) 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 ErrIntOverflowMimir + } + 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: ExemplarRW2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExemplarRW2: 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 ErrIntOverflowMimir + } + 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthMimir + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthMimir + } + 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 ErrIntOverflowMimir + } + 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 ErrIntOverflowMimir + } + 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 := skipMimir(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MetadataRW2) 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 ErrIntOverflowMimir + } + 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: MetadataRW2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MetadataRW2: 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 ErrIntOverflowMimir + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= MetadataRW2_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 ErrIntOverflowMimir + } + 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 ErrIntOverflowMimir + } + 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 := skipMimir(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthMimir + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipMimir(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/pkg/mimirpb/mimir.pb.go.expdiff b/pkg/mimirpb/mimir.pb.go.expdiff index 0ac49f36b5..256901d917 100644 --- a/pkg/mimirpb/mimir.pb.go.expdiff +++ b/pkg/mimirpb/mimir.pb.go.expdiff @@ -1,8 +1,8 @@ diff --git a/pkg/mimirpb/mimir.pb.go b/pkg/mimirpb/mimir.pb.go -index 2fcc5ccde1..0688358568 100644 +index 36005dbcae..d6b7955d30 100644 --- a/pkg/mimirpb/mimir.pb.go +++ b/pkg/mimirpb/mimir.pb.go -@@ -243,9 +243,6 @@ type WriteRequest struct { +@@ -282,9 +282,6 @@ type WriteRequest struct { SkipLabelValidation bool `protobuf:"varint,1000,opt,name=skip_label_validation,json=skipLabelValidation,proto3" json:"skip_label_validation,omitempty"` // Skip label count validation. SkipLabelCountValidation bool `protobuf:"varint,1001,opt,name=skip_label_count_validation,json=skipLabelCountValidation,proto3" json:"skip_label_count_validation,omitempty"` @@ -12,17 +12,17 @@ index 2fcc5ccde1..0688358568 100644 } func (m *WriteRequest) Reset() { *m = WriteRequest{} } -@@ -392,9 +389,6 @@ type TimeSeries struct { - Samples []Sample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples"` - Exemplars []Exemplar `protobuf:"bytes,3,rep,name=exemplars,proto3" json:"exemplars"` - Histograms []Histogram `protobuf:"bytes,4,rep,name=histograms,proto3" json:"histograms"` +@@ -437,9 +434,6 @@ type TimeSeries struct { + // 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"` - - // Skip unmarshaling of exemplars. - SkipUnmarshalingExemplars bool } func (m *TimeSeries) Reset() { *m = TimeSeries{} } -@@ -6212,7 +6206,6 @@ func (m *WriteRequest) Unmarshal(dAtA []byte) error { +@@ -7337,7 +7331,6 @@ func (m *WriteRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } m.Timeseries = append(m.Timeseries, PreallocTimeseries{}) @@ -30,13 +30,13 @@ index 2fcc5ccde1..0688358568 100644 if err := m.Timeseries[len(m.Timeseries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } -@@ -6585,11 +6578,9 @@ func (m *TimeSeries) Unmarshal(dAtA []byte) error { +@@ -7710,11 +7703,9 @@ func (m *TimeSeries) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if !m.SkipUnmarshalingExemplars { -- m.Exemplars = append(m.Exemplars, Exemplar{}) -- if err := m.Exemplars[len(m.Exemplars)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { +- m.Exemplars = append(m.Exemplars, Exemplar{}) +- if err := m.Exemplars[len(m.Exemplars)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Exemplars = append(m.Exemplars, Exemplar{}) diff --git a/pkg/mimirpb/mimir.proto b/pkg/mimirpb/mimir.proto index a22b619cdf..5a2f9492d5 100644 --- a/pkg/mimirpb/mimir.proto +++ b/pkg/mimirpb/mimir.proto @@ -63,6 +63,13 @@ message TimeSeries { repeated Sample samples = 2 [(gogoproto.nullable) = false]; repeated Exemplar exemplars = 3 [(gogoproto.nullable) = false]; repeated Histogram histograms = 4 [(gogoproto.nullable) = false]; + + // Copy from remote write 2.0 + // 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; } message LabelPair { @@ -156,6 +163,9 @@ message Histogram { ResetHint reset_hint = 14; // timestamp is in ms format int64 timestamp = 15; + + // Support Remote Write 2.0 + repeated double custom_values = 16; } // FloatHistogram is based on https://github.com/prometheus/prometheus/blob/main/model/histogram/float_histogram.go. @@ -301,3 +311,102 @@ message MatrixSeries { repeated Sample samples = 2 [(gogoproto.nullable) = false]; repeated FloatHistogramPair histograms = 3 [(gogoproto.nullable) = false]; } + +// Remote write 2.0 support +// RW2.0 Copied and modified from +// https://github.com/prometheus/prometheus/blob/release-3.1/prompb/io/prometheus/write/v2/types.proto + +// WriteRequest1Or2 can unmarshal both Remote Write 1.0 or 2.0 messages. +message WriteRequestRW2 { + repeated string symbols = 4; + repeated TimeSeriesRW2 timeseries = 5 [(gogoproto.nullable) = false]; + + // Mimir-specific fields, using intentionally high field numbers to avoid conflicts with upstream Prometheus. + + // Skip validation of label names and values. + bool skip_label_validation = 1000; + + // Skip label count validation. + bool skip_label_count_validation = 1001; +} + +message TimeSeriesRW2 { + // 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 ExemplarRW2 exemplars = 4 [(gogoproto.nullable) = false]; + + // metadata represents the metadata associated with the given series' samples. + MetadataRW2 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; +} + +message ExemplarRW2 { + // 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; +} + +message MetadataRW2 { + 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; +} diff --git a/pkg/mimirpb/timeseries.go b/pkg/mimirpb/timeseries.go index 3726621775..62f5fe4a3f 100644 --- a/pkg/mimirpb/timeseries.go +++ b/pkg/mimirpb/timeseries.go @@ -63,6 +63,9 @@ type PreallocWriteRequest struct { // SkipUnmarshalingExemplars is an optimization to not unmarshal exemplars when they are disabled by the config anyway. SkipUnmarshalingExemplars bool + + // UnmarshalRW2 is set to true if the Unmarshal method should unmarshal the data as a remote write 2.0 message. + UnmarshalFromRW2 bool } // Unmarshal implements proto.Message. @@ -71,9 +74,200 @@ type PreallocWriteRequest struct { func (p *PreallocWriteRequest) Unmarshal(dAtA []byte) error { p.Timeseries = PreallocTimeseriesSliceFromPool() p.WriteRequest.skipUnmarshalingExemplars = p.SkipUnmarshalingExemplars + + if p.UnmarshalFromRW2 { + return p.unmarshalRW2(dAtA) + } + return p.WriteRequest.Unmarshal(dAtA) } +// UnmarshalRW2 unmarshals the given remote write 2.0 data and converts it to a WriteRequest. +func (p *PreallocWriteRequest) unmarshalRW2(data []byte) error { + rw2req := &WriteRequestRW2{} + if err := rw2req.Unmarshal(data); err != nil { + return err + } + + // For debugging. + metadataReceived := 0 + metadataBytes := 0 + metricFamilies := map[string]*MetricMetadata{} + symbolsInMeta := map[uint32]struct{}{} + + for _, ts := range rw2req.Timeseries { + p.Timeseries = append(p.Timeseries, PreallocTimeseries{}) + p.Timeseries[len(p.Timeseries)-1].TimeSeries = TimeseriesFromPool() + p.Timeseries[len(p.Timeseries)-1].TimeSeries.CreatedTimestamp = ts.CreatedTimestamp + var err error + p.Timeseries[len(p.Timeseries)-1].TimeSeries.Labels, err = labelRefsToLabelAdapter(ts.LabelsRefs, rw2req.Symbols) + if err != nil { + return err + } + p.Timeseries[len(p.Timeseries)-1].TimeSeries.Samples = ts.Samples + p.Timeseries[len(p.Timeseries)-1].TimeSeries.Histograms = ts.Histograms + if !p.SkipUnmarshalingExemplars { + p.Timeseries[len(p.Timeseries)-1].TimeSeries.Exemplars = make([]Exemplar, 0, len(ts.Exemplars)) + for i := range ts.Exemplars { + lbls, err := labelRefsToLabelAdapter(ts.Exemplars[i].LabelsRefs, rw2req.Symbols) + if err != nil { + return err + } + p.Timeseries[len(p.Timeseries)-1].TimeSeries.Exemplars = append(p.Timeseries[len(p.Timeseries)-1].TimeSeries.Exemplars, Exemplar{ + Labels: lbls, + Value: ts.Exemplars[i].Value, + TimestampMs: ts.Exemplars[i].Timestamp, + }) + } + } + + // Convert RW2 metadata to RW1 metadata. + seriesName := getSeriesName(p.Timeseries[len(p.Timeseries)-1].TimeSeries.Labels) + if seriesName == "" { + continue + } + metricFamily, _ := getMetricName(seriesName, ts.Metadata.Type) + if metricFamily == "" { + continue + } + help, _ := getSymbol(ts.Metadata.HelpRef, rw2req.Symbols) + unit, _ := getSymbol(ts.Metadata.UnitRef, rw2req.Symbols) + + { + // Debugging. + buffer := make([]byte, 64*1024) + s, _ := ts.Metadata.MarshalToSizedBuffer(buffer) + metadataBytes += s + + symbolsInMeta[ts.Metadata.HelpRef] = struct{}{} + symbolsInMeta[ts.Metadata.UnitRef] = struct{}{} + } + + if ts.Metadata.Type == METRIC_TYPE_UNSPECIFIED && help == "" && unit == "" { + // Nothing to do here. + continue + } + + { + // Debugging. + metadataReceived++ + } + + metricFamilies[metricFamily] = &MetricMetadata{ + Type: MetricMetadata_MetricType(ts.Metadata.Type), + MetricFamilyName: metricFamily, + Help: help, + Unit: unit, + } + } + + // Fill the metadata + p.Metadata = make([]*MetricMetadata, 0, len(metricFamilies)) + for _, metadata := range metricFamilies { + p.Metadata = append(p.Metadata, metadata) + } + + { + // Debugging + familyBytes := 0 + for metricFamily, metadata := range metricFamilies { + familyBytes += len(metricFamily) + familyBytes += 4 // type + familyBytes += len(metadata.Help) + familyBytes += len(metadata.Unit) + } + + for symbol := range symbolsInMeta { + metadataBytes += len(rw2req.Symbols[symbol]) + } + + fmt.Printf("KRAJO: RW2 timeseries=%v, metadata=%v, series meta=%v bytes, family meta=%v bytes\n", len(p.Timeseries), metadataReceived, metadataBytes, familyBytes) + } + + return nil +} + +// getSymbol resolves the symbol reference to a string. +func getSymbol(ref uint32, symbols []string) (string, error) { + if ref < uint32(len(symbols)) { + return symbols[ref], nil + } + return "", fmt.Errorf("symbol reference %d is out of bounds", ref) +} + +// labelRefsToLabelAdapter converts a slice of label references to a slice +// of LabelAdapter. +func labelRefsToLabelAdapter(refs []uint32, symbols []string) ([]LabelAdapter, error) { + if len(refs)%2 != 0 { + return nil, fmt.Errorf("invalid number of label references: %d", len(refs)) + } + labels := make([]LabelAdapter, 0, len(refs)/2) + for i := 0; i < len(refs); i += 2 { + name, err := getSymbol(refs[i], symbols) + if err != nil { + return nil, err + } + value, err := getSymbol(refs[i+1], symbols) + if err != nil { + return nil, err + } + labels = append(labels, LabelAdapter{Name: name, Value: value}) + } + return labels, nil +} + +// getMetricName cuts the mandatory OpenMetrics suffix from the +// seriesName and returns the metric name and whether it cut the suffix. +// Based on https://github.com/prometheus/OpenMetrics/blob/main/specification/OpenMetrics.md#suffixes +func getMetricName(seriesName string, metricType MetadataRW2_MetricType) (string, bool) { + switch metricType { + case METRIC_TYPE_COUNTER: + return strings.CutSuffix(seriesName, "_total") + case METRIC_TYPE_SUMMARY: + retval, ok := strings.CutSuffix(seriesName, "_count") + if ok { + return retval, true + } + return strings.CutSuffix(seriesName, "_sum") + case METRIC_TYPE_HISTOGRAM: + retval, ok := strings.CutSuffix(seriesName, "_bucket") + if ok { + return retval, true + } + retval, ok = strings.CutSuffix(seriesName, "_count") + if ok { + return retval, true + } + return strings.CutSuffix(seriesName, "_sum") + case METRIC_TYPE_GAUGEHISTOGRAM: + retval, ok := strings.CutSuffix(seriesName, "_bucket") + if ok { + return retval, true + } + retval, ok = strings.CutSuffix(seriesName, "_gcount") + if ok { + return retval, true + } + return strings.CutSuffix(seriesName, "_gsum") + case METRIC_TYPE_INFO: + return strings.CutSuffix(seriesName, "_info") + default: + return seriesName, false + + } +} + +// getSeriesName finds and returns the __name__ label value from the given +// labels. +func getSeriesName(lbls []LabelAdapter) string { + for i := 0; i < len(lbls); i += 2 { + if lbls[i].Name == labels.MetricName { + return lbls[i].Value + } + } + return "" +} + func (p *WriteRequest) ClearTimeseriesUnmarshalData() { for idx := range p.Timeseries { p.Timeseries[idx].clearUnmarshalData()