diff --git a/cmd/tests/cmd_run_test.go b/cmd/tests/cmd_run_test.go index 5e75e8e0cde4..cc6e1f3d0e77 100644 --- a/cmd/tests/cmd_run_test.go +++ b/cmd/tests/cmd_run_test.go @@ -529,6 +529,25 @@ func getCloudTestEndChecker( return srv } +func getCloudMetricsServer(t *testing.T, testRunID int) *httptest.Server { + metricsFlushed := false + testStart := cloudTestStartSimple(t, testRunID) + + srv := getTestServer(t, map[string]http.Handler{ + "POST ^/v1/tests$": testStart, + fmt.Sprintf("POST ^/v2/metrics/%d$", testRunID): http.HandlerFunc(func(resp http.ResponseWriter, req *http.Request) { + metricsFlushed = true + }), + }) + + t.Cleanup(func() { + assert.Truef(t, metricsFlushed, "expected test to have called the cloud API endpoint to flush the metrics") + srv.Close() + }) + + return srv +} + func getSimpleCloudOutputTestState( t *testing.T, script string, cliFlags []string, expRunStatus cloudapi.RunStatus, expResultStatus cloudapi.ResultStatus, expExitCode exitcodes.ExitCode, @@ -1881,3 +1900,43 @@ func TestRunStaticArchives(t *testing.T) { }) } } + +func TestCloudOutputV2(t *testing.T) { + t.Parallel() + script := ` + import { sleep } from 'k6'; + export let options = { + scenarios: { + sc1: { + executor: 'per-vu-iterations', + vus: 1, iterations: 5, + } + }, + thresholds: { + 'iterations': ['count == 5'], + }, + }; + + export default function () { }; + ` + cliFlags := []string{"-v", "--log-output=stdout", "--out", "cloud"} + + srv := getCloudMetricsServer(t, 123) + ts := getSingleFileTestState(t, script, cliFlags, 0) + ts.Env["K6_CLOUD_HOST"] = srv.URL + + cmd.ExecuteWithGlobalState(ts.GlobalState) + + stdout := ts.Stdout.String() + t.Log(stdout) + + assert.Contains(t, stdout, `execution: local`) + assert.Contains(t, stdout, `output: cloud (https://app.k6.io/runs/123)`) + assert.Contains(t, stdout, `Started!" output=cloudv2`) + assert.Contains(t, stdout, `✓ iterations...........: 5`) + assert.Contains(t, stdout, `Successfully flushed buffered samples to the cloud`) + assert.Contains(t, stdout, `Cloud output successfully stopped!`) + assert.Contains(t, stdout, `Stopped!" output=cloudv2`) + + assert.NotContains(t, stdout, `failed to flush metrics`) +} diff --git a/output/cloud/expv2/metrics_client.go b/output/cloud/expv2/metrics_client.go new file mode 100644 index 000000000000..e097413fa801 --- /dev/null +++ b/output/cloud/expv2/metrics_client.go @@ -0,0 +1,106 @@ +package expv2 + +import ( + "bytes" + "context" + "fmt" + "net/http" + "sync" + "time" + + "github.com/golang/snappy" + "github.com/sirupsen/logrus" + "google.golang.org/protobuf/proto" + + "go.k6.io/k6/lib/consts" + "go.k6.io/k6/output/cloud/expv2/pbcloud" +) + +type httpDoer interface { + Do(*http.Request) (*http.Response, error) +} + +// MetricsClient is a wrapper around the cloudapi.Client that is also capable of pushing +type MetricsClient struct { + httpClient httpDoer + logger logrus.FieldLogger + host string + userAgent string + + pushBufferPool sync.Pool +} + +// NewMetricsClient creates and initializes a new MetricsClient. +func NewMetricsClient(logger logrus.FieldLogger, host string) *MetricsClient { + return &MetricsClient{ + httpClient: &http.Client{Timeout: 5 * time.Second}, + logger: logger, + host: host, + userAgent: fmt.Sprintf("k6cloud/v%s", consts.Version), + pushBufferPool: sync.Pool{ + New: func() interface{} { + return &bytes.Buffer{} + }, + }, + } +} + +// Push pushes the provided metric samples for the given referenceID +func (mc *MetricsClient) Push(ctx context.Context, referenceID string, samples *pbcloud.MetricSet) error { + if referenceID == "" { + return fmt.Errorf("a Reference ID of the test run is required") + } + start := time.Now() + url := fmt.Sprintf("%s/v2/metrics/%s", mc.host, referenceID) + + b, err := newRequestBody(samples) + if err != nil { + return err + } + + buf, _ := mc.pushBufferPool.Get().(*bytes.Buffer) + buf.Reset() + defer mc.pushBufferPool.Put(buf) + + _, err = buf.Write(b) + if err != nil { + return err + } + req, err := http.NewRequestWithContext(ctx, http.MethodPost, url, buf) + if err != nil { + return err + } + req.Header.Set("User-Agent", mc.userAgent) + req.Header.Set("Content-Type", "application/x-protobuf") + req.Header.Set("Content-Encoding", "snappy") + req.Header.Set("K6-Metrics-Protocol-Version", "2.0") + + // TODO: Add authentication + // req.Header.Set("Authorization", fmt.Sprintf("Token %s", c.token)) + + resp, err := mc.httpClient.Do(req) + if err != nil { + return fmt.Errorf("failed to push metrics: %w", err) + } + defer func() { + _ = resp.Body.Close() + }() + if resp.StatusCode != http.StatusOK { + return fmt.Errorf("failed to push metrics: push metrics response got an unexpected status code: %s", resp.Status) + } + mc.logger.WithField("t", time.Since(start)).WithField("size", len(b)). + Debug("Pushed part to cloud") + return nil +} + +func newRequestBody(data *pbcloud.MetricSet) ([]byte, error) { + b, err := proto.Marshal(data) + if err != nil { + return nil, fmt.Errorf("encoding series as protobuf write request failed: %w", err) + } + if snappy.MaxEncodedLen(len(b)) < 0 { + return nil, fmt.Errorf("the protobuf message is too large to be handled by Snappy encoder; "+ + "size: %d, limit: %d", len(b), 0xffffffff) + } + return snappy.Encode(nil, b), nil +} diff --git a/output/cloud/expv2/metrics_client_test.go b/output/cloud/expv2/metrics_client_test.go new file mode 100644 index 000000000000..2155f809a16d --- /dev/null +++ b/output/cloud/expv2/metrics_client_test.go @@ -0,0 +1,85 @@ +package expv2 + +import ( + "context" + "fmt" + "io" + "net/http" + "net/http/httptest" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "go.k6.io/k6/lib/testutils" + "go.k6.io/k6/output/cloud/expv2/pbcloud" +) + +type httpDoerFunc func(*http.Request) (*http.Response, error) + +func (fn httpDoerFunc) Do(r *http.Request) (*http.Response, error) { + return fn(r) +} + +func TestMetricsClientPush(t *testing.T) { + t.Parallel() + + done := make(chan struct{}, 1) + reqs := 0 + h := func(rw http.ResponseWriter, r *http.Request) { + defer close(done) + reqs++ + + assert.Equal(t, "/v2/metrics/test-ref-id", r.URL.Path) + assert.Equal(t, http.MethodPost, r.Method) + assert.Contains(t, r.Header.Get("User-Agent"), "k6cloud/v0.4") + assert.Equal(t, "application/x-protobuf", r.Header.Get("Content-Type")) + assert.Equal(t, "snappy", r.Header.Get("Content-Encoding")) + assert.Equal(t, "2.0", r.Header.Get("K6-Metrics-Protocol-Version")) + + b, err := io.ReadAll(r.Body) + require.NoError(t, err) + assert.NotEmpty(t, b) + } + + ts := httptest.NewServer(http.HandlerFunc(h)) + defer ts.Close() + + mc := NewMetricsClient(testutils.NewLogger(t), ts.URL) + mc.httpClient = ts.Client() + + mset := pbcloud.MetricSet{} + err := mc.Push(context.TODO(), "test-ref-id", &mset) + <-done + require.NoError(t, err) + assert.Equal(t, 1, reqs) +} + +func TestMetricsClientPushUnexpectedStatus(t *testing.T) { + t.Parallel() + + h := func(rw http.ResponseWriter, _ *http.Request) { + rw.WriteHeader(http.StatusInternalServerError) + } + ts := httptest.NewServer(http.HandlerFunc(h)) + defer ts.Close() + + mc := NewMetricsClient(testutils.NewLogger(t), ts.URL) + mc.httpClient = ts.Client() + + err := mc.Push(context.TODO(), "test-ref-id", nil) + assert.ErrorContains(t, err, "500 Internal Server Error") +} + +func TestMetricsClientPushError(t *testing.T) { + t.Parallel() + + httpClientMock := func(_ *http.Request) (*http.Response, error) { + return nil, fmt.Errorf("fake generated error") + } + + mc := NewMetricsClient(testutils.NewLogger(t), "") + mc.httpClient = httpDoerFunc(httpClientMock) + + err := mc.Push(context.TODO(), "test-ref-id", nil) + assert.ErrorContains(t, err, "fake generated error") +} diff --git a/output/cloud/expv2/output.go b/output/cloud/expv2/output.go new file mode 100644 index 000000000000..915a04f748b9 --- /dev/null +++ b/output/cloud/expv2/output.go @@ -0,0 +1,217 @@ +// Package expv2 contains a Cloud output using a Protobuf +// binary format for encoding payloads. +package expv2 + +import ( + "context" + "time" + + "github.com/sirupsen/logrus" + "go.k6.io/k6/cloudapi" + "go.k6.io/k6/metrics" + "go.k6.io/k6/output" + "go.k6.io/k6/output/cloud/expv2/pbcloud" + "google.golang.org/protobuf/types/known/timestamppb" +) + +// TestName is the default Cloud test name +const ( + TestName = "k6 test" +) + +// Output sends result data to the k6 Cloud service. +type Output struct { + output.SampleBuffer + + config cloudapi.Config + referenceID string + + logger logrus.FieldLogger + client *MetricsClient + periodicFlusher *output.PeriodicFlusher +} + +// New creates a new cloud output. +func New(logger logrus.FieldLogger, conf cloudapi.Config) (*Output, error) { + return &Output{ + config: conf, + client: NewMetricsClient(logger, conf.Host.String), + logger: logger.WithFields(logrus.Fields{"output": "cloudv2"}), + }, nil +} + +// Start starts the output. +func (o *Output) Start() error { + o.logger.Debug("Starting...") + + // TODO: merge here the part executed by v1 when we will drop it + pf, err := output.NewPeriodicFlusher(o.config.MetricPushInterval.TimeDuration(), o.flushMetrics) + if err != nil { + return err + } + o.logger.Debug("Started!") + o.periodicFlusher = pf + return nil +} + +// StopWithTestError stops the output. +func (o *Output) StopWithTestError(testErr error) error { + o.logger.Debug("Stopping...") + defer o.logger.Debug("Stopped!") + o.periodicFlusher.Stop() + return nil +} + +// SetReferenceID sets the Cloud's test ID. +func (o *Output) SetReferenceID(refID string) { + o.referenceID = refID +} + +// AddMetricSamples receives a set of metric samples. +func (o *Output) flushMetrics() { + if o.referenceID == "" { + // TODO: should it warn? + return + } + + start := time.Now() + + series := o.collectSamples() + if series == nil { + return + } + + metricSet := make([]*pbcloud.Metric, 0, len(series)) + for m, aggr := range series { + metricSet = append(metricSet, o.mapMetricProto(m, aggr)) + } + + ctx, cancel := context.WithTimeout(context.Background(), o.config.MetricPushInterval.TimeDuration()) + defer cancel() + + err := o.client.Push(ctx, o.referenceID, &pbcloud.MetricSet{Metrics: metricSet}) + if err != nil { + o.logger.Error(err) + return + } + + o.logger.WithField("t", time.Since(start)).Debug("Successfully flushed buffered samples to the cloud") +} + +func (o *Output) collectSamples() map[*metrics.Metric]aggregatedSamples { + samplesContainers := o.GetBufferedSamples() + if len(samplesContainers) < 1 { + return nil + } + + // TODO: we expect to do something more complex here + // so a more efficient mapping is expected + + series := make(map[*metrics.Metric]aggregatedSamples) + for _, sampleContainer := range samplesContainers { + samples := sampleContainer.GetSamples() + for _, sample := range samples { + aggr, ok := series[sample.Metric] + if !ok { + aggr = aggregatedSamples{ + Samples: make(map[metrics.TimeSeries][]metrics.Sample), + } + series[sample.Metric] = aggr + } + aggr.AddSample(sample) + } + } + return series +} + +func (o *Output) mapMetricProto(m *metrics.Metric, as aggregatedSamples) *pbcloud.Metric { + var mtype pbcloud.MetricType + switch m.Type { + case metrics.Counter: + mtype = pbcloud.MetricType_COUNTER + case metrics.Gauge: + mtype = pbcloud.MetricType_GAUGE + case metrics.Rate: + mtype = pbcloud.MetricType_RATE + case metrics.Trend: + mtype = pbcloud.MetricType_TREND + } + return &pbcloud.Metric{ + Name: m.Name, + Type: mtype, + TimeSeries: as.MapAsProto(), + } +} + +type aggregatedSamples struct { + Samples map[metrics.TimeSeries][]metrics.Sample +} + +func (as *aggregatedSamples) AddSample(s metrics.Sample) { + ss := as.Samples[s.TimeSeries] + as.Samples[s.TimeSeries] = append(ss, s) +} + +func (as *aggregatedSamples) MapAsProto() []*pbcloud.TimeSeries { + if len(as.Samples) < 1 { + return nil + } + pbseries := make([]*pbcloud.TimeSeries, 0, len(as.Samples)) + for ts, samples := range as.Samples { + pb := pbcloud.TimeSeries{} + // TODO: optimize removing Map + // and using https://github.com/grafana/k6/issues/2764 + for ktag, vtag := range ts.Tags.Map() { + pb.Labels = append(pb.Labels, &pbcloud.Label{Name: ktag, Value: vtag}) + } + + // TODO: extend with other missing types + switch ts.Metric.Type { + case metrics.Counter: + counterSamples := &pbcloud.CounterSamples{} + for _, counterSample := range samples { + counterSamples.Values = append(counterSamples.Values, &pbcloud.CounterValue{ + Time: timestamppb.New(counterSample.Time), + Value: counterSample.Value, + }) + } + pb.Samples = &pbcloud.TimeSeries_CounterSamples{ + CounterSamples: counterSamples, + } + case metrics.Gauge: + gaugeSamples := &pbcloud.GaugeSamples{} + for _, gaugeSample := range samples { + gaugeSamples.Values = append(gaugeSamples.Values, &pbcloud.GaugeValue{ + Time: timestamppb.New(gaugeSample.Time), + Last: gaugeSample.Value, + Min: gaugeSample.Value, + Max: gaugeSample.Value, + Avg: gaugeSample.Value, + }) + } + pb.Samples = &pbcloud.TimeSeries_GaugeSamples{ + GaugeSamples: gaugeSamples, + } + case metrics.Rate: + rateSamples := &pbcloud.RateSamples{} + for _, rateSample := range samples { + nonzero := uint32(0) + if rateSample.Value != 0 { + nonzero = 1 + } + rateSamples.Values = append(rateSamples.Values, &pbcloud.RateValue{ + Time: timestamppb.New(rateSample.Time), + NonzeroCount: nonzero, + TotalCount: 1, + }) + } + pb.Samples = &pbcloud.TimeSeries_RateSamples{ + RateSamples: rateSamples, + } + case metrics.Trend: + // TODO: implement the HDR histogram mapping + } + pbseries = append(pbseries, &pb) + } + return pbseries +} diff --git a/output/cloud/expv2/output_test.go b/output/cloud/expv2/output_test.go new file mode 100644 index 000000000000..c7ce34b83c8d --- /dev/null +++ b/output/cloud/expv2/output_test.go @@ -0,0 +1,163 @@ +package expv2 + +import ( + "fmt" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "go.k6.io/k6/metrics" + "go.k6.io/k6/output/cloud/expv2/pbcloud" + "google.golang.org/protobuf/types/known/timestamppb" +) + +func TestOutputCollectSamples(t *testing.T) { + t.Parallel() + + o := Output{} + r := metrics.NewRegistry() + + m1 := r.MustNewMetric("metric1", metrics.Counter) + s1 := metrics.Sample{ + TimeSeries: metrics.TimeSeries{ + Metric: m1, + Tags: r.RootTagSet().With("key1", "val1"), + }, + } + subs1 := metrics.Sample{ + TimeSeries: metrics.TimeSeries{ + Metric: m1, + Tags: r.RootTagSet().With("key1", "valsub1"), + }, + } + m2 := r.MustNewMetric("metric2", metrics.Counter) + s2 := metrics.Sample{ + TimeSeries: metrics.TimeSeries{ + Metric: m2, + Tags: r.RootTagSet().With("key2", "val2"), + }, + } + + o.AddMetricSamples([]metrics.SampleContainer{ + metrics.Samples{s1}, + metrics.Samples{s2}, + metrics.Samples{subs1}, + metrics.Samples{s1}, + }) + + samplesByMetrics := o.collectSamples() + require.Len(t, samplesByMetrics, 2) + assert.Equal(t, []metrics.Sample{s1, s1}, samplesByMetrics[m1].Samples[s1.TimeSeries]) + assert.Equal(t, []metrics.Sample{subs1}, samplesByMetrics[m1].Samples[subs1.TimeSeries]) + assert.Equal(t, []metrics.Sample{s2}, samplesByMetrics[m2].Samples[s2.TimeSeries]) +} + +func TestOutputMapMetricProto(t *testing.T) { + t.Parallel() + + o := Output{} + r := metrics.NewRegistry() + + m1 := r.MustNewMetric("metric1", metrics.Counter) + s1 := metrics.Sample{ + TimeSeries: metrics.TimeSeries{ + Metric: m1, + Tags: r.RootTagSet().With("key1", "val1"), + }, + } + + aggSamples := aggregatedSamples{ + Samples: map[metrics.TimeSeries][]metrics.Sample{ + s1.TimeSeries: {s1}, + }, + } + + protodata := o.mapMetricProto(m1, aggSamples) + assert.Equal(t, "metric1", protodata.Name) + assert.Equal(t, "COUNTER", pbcloud.MetricType_name[int32(protodata.Type)]) + assert.Len(t, protodata.TimeSeries, 1) +} + +func TestAggregatedSamplesMapAsProto(t *testing.T) { + t.Parallel() + + expLabels := []*pbcloud.Label{{Name: "key1", Value: "val1"}} + now := time.Now() + tests := []struct { + mtyp metrics.MetricType + expmap *pbcloud.TimeSeries + }{ + { + mtyp: metrics.Counter, + expmap: &pbcloud.TimeSeries{ + Labels: expLabels, + Samples: &pbcloud.TimeSeries_CounterSamples{ + CounterSamples: &pbcloud.CounterSamples{ + Values: []*pbcloud.CounterValue{ + {Time: timestamppb.New(now), Value: 42}, + {Time: timestamppb.New(now), Value: 42}, + }, + }, + }, + }, + }, + { + mtyp: metrics.Gauge, + expmap: &pbcloud.TimeSeries{ + Labels: expLabels, + Samples: &pbcloud.TimeSeries_GaugeSamples{ + GaugeSamples: &pbcloud.GaugeSamples{ + Values: []*pbcloud.GaugeValue{ + {Time: timestamppb.New(now), Last: 42, Min: 42, Max: 42, Avg: 42}, + {Time: timestamppb.New(now), Last: 42, Min: 42, Max: 42, Avg: 42}, + }, + }, + }, + }, + }, + { + mtyp: metrics.Rate, + expmap: &pbcloud.TimeSeries{ + Labels: expLabels, + Samples: &pbcloud.TimeSeries_RateSamples{ + RateSamples: &pbcloud.RateSamples{ + Values: []*pbcloud.RateValue{ + {Time: timestamppb.New(now), NonzeroCount: 1, TotalCount: 1}, + {Time: timestamppb.New(now), NonzeroCount: 1, TotalCount: 1}, + }, + }, + }, + }, + }, + // {mtyp: metrics.Trend}, + } + + r := metrics.NewRegistry() + + for _, tc := range tests { + tc := tc + t.Run(tc.mtyp.String(), func(t *testing.T) { + t.Parallel() + + s1 := metrics.Sample{ + TimeSeries: metrics.TimeSeries{ + Metric: r.MustNewMetric(fmt.Sprintf("metric%s", tc.mtyp.String()), tc.mtyp), + Tags: r.RootTagSet().With("key1", "val1"), + }, + Time: now, + Value: 42.0, + } + + aggSamples := aggregatedSamples{ + Samples: map[metrics.TimeSeries][]metrics.Sample{ + s1.TimeSeries: {s1, s1}, + }, + } + pbsamples := aggSamples.MapAsProto() + require.Len(t, pbsamples, 1) + assert.Equal(t, tc.expmap.Labels, pbsamples[0].Labels) + assert.Equal(t, tc.expmap.Samples, pbsamples[0].Samples) + }) + } +} diff --git a/output/cloud/expv2/pbcloud/metric.pb.go b/output/cloud/expv2/pbcloud/metric.pb.go new file mode 100644 index 000000000000..3c5134dbe44e --- /dev/null +++ b/output/cloud/expv2/pbcloud/metric.pb.go @@ -0,0 +1,1270 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.28.1 +// protoc v3.21.12 +// source: metric.proto + +package pbcloud + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// The type of a Metric. +type MetricType int32 + +const ( + MetricType_METRICTYPE_UNSPECIFIED MetricType = 0 + MetricType_COUNTER MetricType = 1 + MetricType_GAUGE MetricType = 2 + MetricType_RATE MetricType = 3 + MetricType_TREND MetricType = 4 +) + +// Enum value maps for MetricType. +var ( + MetricType_name = map[int32]string{ + 0: "METRICTYPE_UNSPECIFIED", + 1: "COUNTER", + 2: "GAUGE", + 3: "RATE", + 4: "TREND", + } + MetricType_value = map[string]int32{ + "METRICTYPE_UNSPECIFIED": 0, + "COUNTER": 1, + "GAUGE": 2, + "RATE": 3, + "TREND": 4, + } +) + +func (x MetricType) Enum() *MetricType { + p := new(MetricType) + *p = x + return p +} + +func (x MetricType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (MetricType) Descriptor() protoreflect.EnumDescriptor { + return file_metric_proto_enumTypes[0].Descriptor() +} + +func (MetricType) Type() protoreflect.EnumType { + return &file_metric_proto_enumTypes[0] +} + +func (x MetricType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use MetricType.Descriptor instead. +func (MetricType) EnumDescriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{0} +} + +// MetricSet is the top-level container type +// that is encoded and sent over the wire. +type MetricSet struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Metrics []*Metric `protobuf:"bytes,1,rep,name=metrics,proto3" json:"metrics,omitempty"` +} + +func (x *MetricSet) Reset() { + *x = MetricSet{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MetricSet) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MetricSet) ProtoMessage() {} + +func (x *MetricSet) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MetricSet.ProtoReflect.Descriptor instead. +func (*MetricSet) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{0} +} + +func (x *MetricSet) GetMetrics() []*Metric { + if x != nil { + return x.Metrics + } + return nil +} + +type Metric struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Required. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Required. + Type MetricType `protobuf:"varint,2,opt,name=type,proto3,enum=metrics.MetricType" json:"type,omitempty"` + // Optional. + TimeSeries []*TimeSeries `protobuf:"bytes,3,rep,name=time_series,json=timeSeries,proto3" json:"time_series,omitempty"` +} + +func (x *Metric) Reset() { + *x = Metric{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Metric) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Metric) ProtoMessage() {} + +func (x *Metric) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Metric.ProtoReflect.Descriptor instead. +func (*Metric) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{1} +} + +func (x *Metric) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Metric) GetType() MetricType { + if x != nil { + return x.Type + } + return MetricType_METRICTYPE_UNSPECIFIED +} + +func (x *Metric) GetTimeSeries() []*TimeSeries { + if x != nil { + return x.TimeSeries + } + return nil +} + +// Label is a name-value pair. +type Label struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Required. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Required. + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` +} + +func (x *Label) Reset() { + *x = Label{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Label) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Label) ProtoMessage() {} + +func (x *Label) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Label.ProtoReflect.Descriptor instead. +func (*Label) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{2} +} + +func (x *Label) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Label) GetValue() string { + if x != nil { + return x.Value + } + return "" +} + +// TimeSeries is a single metric with a unique set of labels.. +type TimeSeries struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Labels []*Label `protobuf:"bytes,1,rep,name=labels,proto3" json:"labels,omitempty"` + // time bucket size in seconds + AggregationPeriod uint32 `protobuf:"varint,2,opt,name=aggregation_period,json=aggregationPeriod,proto3" json:"aggregation_period,omitempty"` + // Types that are assignable to Samples: + // + // *TimeSeries_CounterSamples + // *TimeSeries_GaugeSamples + // *TimeSeries_RateSamples + // *TimeSeries_TrendHdrSamples + Samples isTimeSeries_Samples `protobuf_oneof:"samples"` +} + +func (x *TimeSeries) Reset() { + *x = TimeSeries{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimeSeries) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimeSeries) ProtoMessage() {} + +func (x *TimeSeries) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimeSeries.ProtoReflect.Descriptor instead. +func (*TimeSeries) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{3} +} + +func (x *TimeSeries) GetLabels() []*Label { + if x != nil { + return x.Labels + } + return nil +} + +func (x *TimeSeries) GetAggregationPeriod() uint32 { + if x != nil { + return x.AggregationPeriod + } + return 0 +} + +func (m *TimeSeries) GetSamples() isTimeSeries_Samples { + if m != nil { + return m.Samples + } + return nil +} + +func (x *TimeSeries) GetCounterSamples() *CounterSamples { + if x, ok := x.GetSamples().(*TimeSeries_CounterSamples); ok { + return x.CounterSamples + } + return nil +} + +func (x *TimeSeries) GetGaugeSamples() *GaugeSamples { + if x, ok := x.GetSamples().(*TimeSeries_GaugeSamples); ok { + return x.GaugeSamples + } + return nil +} + +func (x *TimeSeries) GetRateSamples() *RateSamples { + if x, ok := x.GetSamples().(*TimeSeries_RateSamples); ok { + return x.RateSamples + } + return nil +} + +func (x *TimeSeries) GetTrendHdrSamples() *TrendHdrSamples { + if x, ok := x.GetSamples().(*TimeSeries_TrendHdrSamples); ok { + return x.TrendHdrSamples + } + return nil +} + +type isTimeSeries_Samples interface { + isTimeSeries_Samples() +} + +type TimeSeries_CounterSamples struct { + CounterSamples *CounterSamples `protobuf:"bytes,3,opt,name=counter_samples,json=counterSamples,proto3,oneof"` +} + +type TimeSeries_GaugeSamples struct { + GaugeSamples *GaugeSamples `protobuf:"bytes,4,opt,name=gauge_samples,json=gaugeSamples,proto3,oneof"` +} + +type TimeSeries_RateSamples struct { + RateSamples *RateSamples `protobuf:"bytes,5,opt,name=rate_samples,json=rateSamples,proto3,oneof"` +} + +type TimeSeries_TrendHdrSamples struct { + TrendHdrSamples *TrendHdrSamples `protobuf:"bytes,6,opt,name=trend_hdr_samples,json=trendHdrSamples,proto3,oneof"` +} + +func (*TimeSeries_CounterSamples) isTimeSeries_Samples() {} + +func (*TimeSeries_GaugeSamples) isTimeSeries_Samples() {} + +func (*TimeSeries_RateSamples) isTimeSeries_Samples() {} + +func (*TimeSeries_TrendHdrSamples) isTimeSeries_Samples() {} + +type CounterSamples struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Values []*CounterValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` +} + +func (x *CounterSamples) Reset() { + *x = CounterSamples{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CounterSamples) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CounterSamples) ProtoMessage() {} + +func (x *CounterSamples) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CounterSamples.ProtoReflect.Descriptor instead. +func (*CounterSamples) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{4} +} + +func (x *CounterSamples) GetValues() []*CounterValue { + if x != nil { + return x.Values + } + return nil +} + +type GaugeSamples struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Values []*GaugeValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` +} + +func (x *GaugeSamples) Reset() { + *x = GaugeSamples{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GaugeSamples) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GaugeSamples) ProtoMessage() {} + +func (x *GaugeSamples) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GaugeSamples.ProtoReflect.Descriptor instead. +func (*GaugeSamples) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{5} +} + +func (x *GaugeSamples) GetValues() []*GaugeValue { + if x != nil { + return x.Values + } + return nil +} + +type RateSamples struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Values []*RateValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` +} + +func (x *RateSamples) Reset() { + *x = RateSamples{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RateSamples) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RateSamples) ProtoMessage() {} + +func (x *RateSamples) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RateSamples.ProtoReflect.Descriptor instead. +func (*RateSamples) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{6} +} + +func (x *RateSamples) GetValues() []*RateValue { + if x != nil { + return x.Values + } + return nil +} + +type TrendHdrSamples struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Values []*TrendHdrValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` +} + +func (x *TrendHdrSamples) Reset() { + *x = TrendHdrSamples{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TrendHdrSamples) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TrendHdrSamples) ProtoMessage() {} + +func (x *TrendHdrSamples) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TrendHdrSamples.ProtoReflect.Descriptor instead. +func (*TrendHdrSamples) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{7} +} + +func (x *TrendHdrSamples) GetValues() []*TrendHdrValue { + if x != nil { + return x.Values + } + return nil +} + +type CounterValue struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Time *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=time,proto3" json:"time,omitempty"` + Value float64 `protobuf:"fixed64,2,opt,name=value,proto3" json:"value,omitempty"` +} + +func (x *CounterValue) Reset() { + *x = CounterValue{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CounterValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CounterValue) ProtoMessage() {} + +func (x *CounterValue) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CounterValue.ProtoReflect.Descriptor instead. +func (*CounterValue) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{8} +} + +func (x *CounterValue) GetTime() *timestamppb.Timestamp { + if x != nil { + return x.Time + } + return nil +} + +func (x *CounterValue) GetValue() float64 { + if x != nil { + return x.Value + } + return 0 +} + +type GaugeValue struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Time *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=time,proto3" json:"time,omitempty"` + // last observed value + Last float64 `protobuf:"fixed64,2,opt,name=last,proto3" json:"last,omitempty"` + // count of observations + Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` + // min, max, and average of observed values + Min float64 `protobuf:"fixed64,4,opt,name=min,proto3" json:"min,omitempty"` + Max float64 `protobuf:"fixed64,5,opt,name=max,proto3" json:"max,omitempty"` + Avg float64 `protobuf:"fixed64,6,opt,name=avg,proto3" json:"avg,omitempty"` +} + +func (x *GaugeValue) Reset() { + *x = GaugeValue{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GaugeValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GaugeValue) ProtoMessage() {} + +func (x *GaugeValue) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GaugeValue.ProtoReflect.Descriptor instead. +func (*GaugeValue) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{9} +} + +func (x *GaugeValue) GetTime() *timestamppb.Timestamp { + if x != nil { + return x.Time + } + return nil +} + +func (x *GaugeValue) GetLast() float64 { + if x != nil { + return x.Last + } + return 0 +} + +func (x *GaugeValue) GetCount() uint32 { + if x != nil { + return x.Count + } + return 0 +} + +func (x *GaugeValue) GetMin() float64 { + if x != nil { + return x.Min + } + return 0 +} + +func (x *GaugeValue) GetMax() float64 { + if x != nil { + return x.Max + } + return 0 +} + +func (x *GaugeValue) GetAvg() float64 { + if x != nil { + return x.Avg + } + return 0 +} + +type RateValue struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Time *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=time,proto3" json:"time,omitempty"` + NonzeroCount uint32 `protobuf:"varint,2,opt,name=nonzero_count,json=nonzeroCount,proto3" json:"nonzero_count,omitempty"` + TotalCount uint32 `protobuf:"varint,3,opt,name=total_count,json=totalCount,proto3" json:"total_count,omitempty"` +} + +func (x *RateValue) Reset() { + *x = RateValue{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RateValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RateValue) ProtoMessage() {} + +func (x *RateValue) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RateValue.ProtoReflect.Descriptor instead. +func (*RateValue) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{10} +} + +func (x *RateValue) GetTime() *timestamppb.Timestamp { + if x != nil { + return x.Time + } + return nil +} + +func (x *RateValue) GetNonzeroCount() uint32 { + if x != nil { + return x.NonzeroCount + } + return 0 +} + +func (x *RateValue) GetTotalCount() uint32 { + if x != nil { + return x.TotalCount + } + return 0 +} + +type TrendHdrValue struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Time *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=time,proto3" json:"time,omitempty"` + // histogram parameter - value multiplier aka smallest value, default = 1.0 + MinResolution float64 `protobuf:"fixed64,2,opt,name=min_resolution,json=minResolution,proto3" json:"min_resolution,omitempty"` + // histogram parameter - number of significant digits used to calculate buckets formula, + // default = 2 + SignificantDigits uint32 `protobuf:"varint,3,opt,name=significant_digits,json=significantDigits,proto3" json:"significant_digits,omitempty"` + // buckets + Counters []uint32 `protobuf:"varint,4,rep,packed,name=counters,proto3" json:"counters,omitempty"` + // index of first bucket in `counters` + LowerCounterIndex uint32 `protobuf:"varint,5,opt,name=lower_counter_index,json=lowerCounterIndex,proto3" json:"lower_counter_index,omitempty"` + // sum of observations + Sum float64 `protobuf:"fixed64,6,opt,name=sum,proto3" json:"sum,omitempty"` + // count of observations + Count uint32 `protobuf:"varint,7,opt,name=count,proto3" json:"count,omitempty"` + // smallest and largest observed value + MinValue float64 `protobuf:"fixed64,8,opt,name=min_value,json=minValue,proto3" json:"min_value,omitempty"` + MaxValue float64 `protobuf:"fixed64,9,opt,name=max_value,json=maxValue,proto3" json:"max_value,omitempty"` + // counters for zero- and infinity-buckets + ExtraLowValuesCounter *uint32 `protobuf:"varint,10,opt,name=extra_low_values_counter,json=extraLowValuesCounter,proto3,oneof" json:"extra_low_values_counter,omitempty"` + ExtraHighValuesCounter *uint32 `protobuf:"varint,11,opt,name=extra_high_values_counter,json=extraHighValuesCounter,proto3,oneof" json:"extra_high_values_counter,omitempty"` +} + +func (x *TrendHdrValue) Reset() { + *x = TrendHdrValue{} + if protoimpl.UnsafeEnabled { + mi := &file_metric_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TrendHdrValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TrendHdrValue) ProtoMessage() {} + +func (x *TrendHdrValue) ProtoReflect() protoreflect.Message { + mi := &file_metric_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TrendHdrValue.ProtoReflect.Descriptor instead. +func (*TrendHdrValue) Descriptor() ([]byte, []int) { + return file_metric_proto_rawDescGZIP(), []int{11} +} + +func (x *TrendHdrValue) GetTime() *timestamppb.Timestamp { + if x != nil { + return x.Time + } + return nil +} + +func (x *TrendHdrValue) GetMinResolution() float64 { + if x != nil { + return x.MinResolution + } + return 0 +} + +func (x *TrendHdrValue) GetSignificantDigits() uint32 { + if x != nil { + return x.SignificantDigits + } + return 0 +} + +func (x *TrendHdrValue) GetCounters() []uint32 { + if x != nil { + return x.Counters + } + return nil +} + +func (x *TrendHdrValue) GetLowerCounterIndex() uint32 { + if x != nil { + return x.LowerCounterIndex + } + return 0 +} + +func (x *TrendHdrValue) GetSum() float64 { + if x != nil { + return x.Sum + } + return 0 +} + +func (x *TrendHdrValue) GetCount() uint32 { + if x != nil { + return x.Count + } + return 0 +} + +func (x *TrendHdrValue) GetMinValue() float64 { + if x != nil { + return x.MinValue + } + return 0 +} + +func (x *TrendHdrValue) GetMaxValue() float64 { + if x != nil { + return x.MaxValue + } + return 0 +} + +func (x *TrendHdrValue) GetExtraLowValuesCounter() uint32 { + if x != nil && x.ExtraLowValuesCounter != nil { + return *x.ExtraLowValuesCounter + } + return 0 +} + +func (x *TrendHdrValue) GetExtraHighValuesCounter() uint32 { + if x != nil && x.ExtraHighValuesCounter != nil { + return *x.ExtraHighValuesCounter + } + return 0 +} + +var File_metric_proto protoreflect.FileDescriptor + +var file_metric_proto_rawDesc = []byte{ + 0x0a, 0x0c, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x07, + 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x36, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x53, 0x65, 0x74, 0x12, 0x29, 0x0a, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, + 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, + 0x22, 0x7b, 0x0a, 0x06, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x27, + 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x5f, + 0x73, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x69, 0x65, + 0x73, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x69, 0x65, 0x73, 0x22, 0x31, 0x0a, + 0x05, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x22, 0xf3, 0x02, 0x0a, 0x0a, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, + 0x26, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x0e, 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x52, + 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x2d, 0x0a, 0x12, 0x61, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x11, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x50, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x12, 0x42, 0x0a, 0x0f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, + 0x72, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x17, 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, + 0x72, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x65, 0x72, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x0d, 0x67, 0x61, + 0x75, 0x67, 0x65, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x15, 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x47, 0x61, 0x75, 0x67, + 0x65, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x48, 0x00, 0x52, 0x0c, 0x67, 0x61, 0x75, 0x67, + 0x65, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x12, 0x39, 0x0a, 0x0c, 0x72, 0x61, 0x74, 0x65, + 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, + 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x52, 0x61, 0x74, 0x65, 0x53, 0x61, 0x6d, + 0x70, 0x6c, 0x65, 0x73, 0x48, 0x00, 0x52, 0x0b, 0x72, 0x61, 0x74, 0x65, 0x53, 0x61, 0x6d, 0x70, + 0x6c, 0x65, 0x73, 0x12, 0x46, 0x0a, 0x11, 0x74, 0x72, 0x65, 0x6e, 0x64, 0x5f, 0x68, 0x64, 0x72, + 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, + 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x54, 0x72, 0x65, 0x6e, 0x64, 0x48, 0x64, + 0x72, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x74, 0x72, 0x65, 0x6e, + 0x64, 0x48, 0x64, 0x72, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x42, 0x09, 0x0a, 0x07, 0x73, + 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x22, 0x3f, 0x0a, 0x0e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, + 0x72, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x12, 0x2d, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x2e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, + 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x3b, 0x0a, 0x0c, 0x47, 0x61, 0x75, 0x67, 0x65, + 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x2e, 0x47, 0x61, 0x75, 0x67, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x73, 0x22, 0x39, 0x0a, 0x0b, 0x52, 0x61, 0x74, 0x65, 0x53, 0x61, 0x6d, 0x70, + 0x6c, 0x65, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x52, 0x61, + 0x74, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, + 0x41, 0x0a, 0x0f, 0x54, 0x72, 0x65, 0x6e, 0x64, 0x48, 0x64, 0x72, 0x53, 0x61, 0x6d, 0x70, 0x6c, + 0x65, 0x73, 0x12, 0x2e, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x54, 0x72, 0x65, + 0x6e, 0x64, 0x48, 0x64, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x73, 0x22, 0x54, 0x0a, 0x0c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x12, 0x2e, 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x04, 0x74, 0x69, + 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x9c, 0x01, 0x0a, 0x0a, 0x47, 0x61, 0x75, + 0x67, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2e, 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x61, 0x73, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x04, 0x6c, 0x61, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x69, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x03, + 0x6d, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x61, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, + 0x52, 0x03, 0x6d, 0x61, 0x78, 0x12, 0x10, 0x0a, 0x03, 0x61, 0x76, 0x67, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x01, 0x52, 0x03, 0x61, 0x76, 0x67, 0x22, 0x81, 0x01, 0x0a, 0x09, 0x52, 0x61, 0x74, 0x65, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2e, 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x04, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x6f, 0x6e, 0x7a, 0x65, 0x72, 0x6f, + 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6e, 0x6f, + 0x6e, 0x7a, 0x65, 0x72, 0x6f, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, + 0x74, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xfc, 0x03, 0x0a, 0x0d, + 0x54, 0x72, 0x65, 0x6e, 0x64, 0x48, 0x64, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2e, 0x0a, + 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x25, 0x0a, + 0x0e, 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0d, 0x6d, 0x69, 0x6e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x12, 0x73, 0x69, 0x67, 0x6e, 0x69, 0x66, 0x69, 0x63, + 0x61, 0x6e, 0x74, 0x5f, 0x64, 0x69, 0x67, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x11, 0x73, 0x69, 0x67, 0x6e, 0x69, 0x66, 0x69, 0x63, 0x61, 0x6e, 0x74, 0x44, 0x69, 0x67, + 0x69, 0x74, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x08, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x12, + 0x2e, 0x0a, 0x13, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, + 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x11, 0x6c, 0x6f, + 0x77, 0x65, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x10, 0x0a, 0x03, 0x73, 0x75, 0x6d, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x52, 0x03, 0x73, 0x75, + 0x6d, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x6d, 0x69, 0x6e, 0x5f, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x01, 0x52, 0x08, 0x6d, 0x69, 0x6e, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x6d, 0x61, 0x78, 0x5f, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x01, 0x52, 0x08, 0x6d, 0x61, 0x78, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x12, 0x3c, 0x0a, 0x18, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x6c, 0x6f, 0x77, 0x5f, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x0a, 0x20, + 0x01, 0x28, 0x0d, 0x48, 0x00, 0x52, 0x15, 0x65, 0x78, 0x74, 0x72, 0x61, 0x4c, 0x6f, 0x77, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x88, 0x01, 0x01, 0x12, + 0x3e, 0x0a, 0x19, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x68, 0x69, 0x67, 0x68, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x0d, 0x48, 0x01, 0x52, 0x16, 0x65, 0x78, 0x74, 0x72, 0x61, 0x48, 0x69, 0x67, 0x68, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x88, 0x01, 0x01, 0x42, + 0x1b, 0x0a, 0x19, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x6c, 0x6f, 0x77, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x42, 0x1c, 0x0a, 0x1a, + 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x68, 0x69, 0x67, 0x68, 0x5f, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x2a, 0x55, 0x0a, 0x0a, 0x4d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x4d, 0x45, 0x54, 0x52, + 0x49, 0x43, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, + 0x45, 0x44, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x45, 0x52, 0x10, + 0x01, 0x12, 0x09, 0x0a, 0x05, 0x47, 0x41, 0x55, 0x47, 0x45, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, + 0x52, 0x41, 0x54, 0x45, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x52, 0x45, 0x4e, 0x44, 0x10, + 0x04, 0x42, 0x27, 0x5a, 0x25, 0x67, 0x6f, 0x2e, 0x6b, 0x36, 0x2e, 0x69, 0x6f, 0x2f, 0x6b, 0x36, + 0x2f, 0x6f, 0x74, 0x70, 0x75, 0x74, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x65, 0x78, 0x70, + 0x76, 0x32, 0x2f, 0x70, 0x62, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, +} + +var ( + file_metric_proto_rawDescOnce sync.Once + file_metric_proto_rawDescData = file_metric_proto_rawDesc +) + +func file_metric_proto_rawDescGZIP() []byte { + file_metric_proto_rawDescOnce.Do(func() { + file_metric_proto_rawDescData = protoimpl.X.CompressGZIP(file_metric_proto_rawDescData) + }) + return file_metric_proto_rawDescData +} + +var file_metric_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_metric_proto_msgTypes = make([]protoimpl.MessageInfo, 12) +var file_metric_proto_goTypes = []interface{}{ + (MetricType)(0), // 0: metrics.MetricType + (*MetricSet)(nil), // 1: metrics.MetricSet + (*Metric)(nil), // 2: metrics.Metric + (*Label)(nil), // 3: metrics.Label + (*TimeSeries)(nil), // 4: metrics.TimeSeries + (*CounterSamples)(nil), // 5: metrics.CounterSamples + (*GaugeSamples)(nil), // 6: metrics.GaugeSamples + (*RateSamples)(nil), // 7: metrics.RateSamples + (*TrendHdrSamples)(nil), // 8: metrics.TrendHdrSamples + (*CounterValue)(nil), // 9: metrics.CounterValue + (*GaugeValue)(nil), // 10: metrics.GaugeValue + (*RateValue)(nil), // 11: metrics.RateValue + (*TrendHdrValue)(nil), // 12: metrics.TrendHdrValue + (*timestamppb.Timestamp)(nil), // 13: google.protobuf.Timestamp +} +var file_metric_proto_depIdxs = []int32{ + 2, // 0: metrics.MetricSet.metrics:type_name -> metrics.Metric + 0, // 1: metrics.Metric.type:type_name -> metrics.MetricType + 4, // 2: metrics.Metric.time_series:type_name -> metrics.TimeSeries + 3, // 3: metrics.TimeSeries.labels:type_name -> metrics.Label + 5, // 4: metrics.TimeSeries.counter_samples:type_name -> metrics.CounterSamples + 6, // 5: metrics.TimeSeries.gauge_samples:type_name -> metrics.GaugeSamples + 7, // 6: metrics.TimeSeries.rate_samples:type_name -> metrics.RateSamples + 8, // 7: metrics.TimeSeries.trend_hdr_samples:type_name -> metrics.TrendHdrSamples + 9, // 8: metrics.CounterSamples.values:type_name -> metrics.CounterValue + 10, // 9: metrics.GaugeSamples.values:type_name -> metrics.GaugeValue + 11, // 10: metrics.RateSamples.values:type_name -> metrics.RateValue + 12, // 11: metrics.TrendHdrSamples.values:type_name -> metrics.TrendHdrValue + 13, // 12: metrics.CounterValue.time:type_name -> google.protobuf.Timestamp + 13, // 13: metrics.GaugeValue.time:type_name -> google.protobuf.Timestamp + 13, // 14: metrics.RateValue.time:type_name -> google.protobuf.Timestamp + 13, // 15: metrics.TrendHdrValue.time:type_name -> google.protobuf.Timestamp + 16, // [16:16] is the sub-list for method output_type + 16, // [16:16] is the sub-list for method input_type + 16, // [16:16] is the sub-list for extension type_name + 16, // [16:16] is the sub-list for extension extendee + 0, // [0:16] is the sub-list for field type_name +} + +func init() { file_metric_proto_init() } +func file_metric_proto_init() { + if File_metric_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_metric_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MetricSet); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Metric); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Label); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TimeSeries); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CounterSamples); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GaugeSamples); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RateSamples); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TrendHdrSamples); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CounterValue); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GaugeValue); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RateValue); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_metric_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TrendHdrValue); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_metric_proto_msgTypes[3].OneofWrappers = []interface{}{ + (*TimeSeries_CounterSamples)(nil), + (*TimeSeries_GaugeSamples)(nil), + (*TimeSeries_RateSamples)(nil), + (*TimeSeries_TrendHdrSamples)(nil), + } + file_metric_proto_msgTypes[11].OneofWrappers = []interface{}{} + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_metric_proto_rawDesc, + NumEnums: 1, + NumMessages: 12, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_metric_proto_goTypes, + DependencyIndexes: file_metric_proto_depIdxs, + EnumInfos: file_metric_proto_enumTypes, + MessageInfos: file_metric_proto_msgTypes, + }.Build() + File_metric_proto = out.File + file_metric_proto_rawDesc = nil + file_metric_proto_goTypes = nil + file_metric_proto_depIdxs = nil +} diff --git a/output/cloud/expv2/pbcloud/metric.proto b/output/cloud/expv2/pbcloud/metric.proto new file mode 100644 index 000000000000..e38ca910cf87 --- /dev/null +++ b/output/cloud/expv2/pbcloud/metric.proto @@ -0,0 +1,123 @@ +syntax = "proto3"; + +package metrics; + +option go_package = "go.k6.io/k6/otput/cloud/expv2/pbcloud"; + +import "google/protobuf/timestamp.proto"; + +// The type of a Metric. +enum MetricType { + METRICTYPE_UNSPECIFIED = 0; + COUNTER = 1; + GAUGE = 2; + RATE = 3; + TREND = 4; +} + +// MetricSet is the top-level container type +// that is encoded and sent over the wire. +message MetricSet { + repeated Metric metrics = 1; +} + +message Metric { + // Required. + string name = 1; + + // Required. + MetricType type = 2; + + // Optional. + repeated TimeSeries time_series = 3; +} + +// Label is a name-value pair. +message Label { + // Required. + string name = 1; + + // Required. + string value = 2; +} + +// TimeSeries is a single metric with a unique set of labels.. +message TimeSeries { + repeated Label labels = 1; + + // time bucket size in seconds + uint32 aggregation_period = 2; + + oneof samples { + CounterSamples counter_samples = 3; + GaugeSamples gauge_samples = 4; + RateSamples rate_samples = 5; + TrendHdrSamples trend_hdr_samples = 6; + } +} + +message CounterSamples { + repeated CounterValue values = 1; +} + +message GaugeSamples { + repeated GaugeValue values = 1; +} + +message RateSamples { + repeated RateValue values = 1; +} + +message TrendHdrSamples { + repeated TrendHdrValue values = 1; +} + +message CounterValue { + google.protobuf.Timestamp time = 1; + double value = 2; +} + +message GaugeValue { + google.protobuf.Timestamp time = 1; + + // last observed value + double last = 2; + // count of observations + uint32 count = 3; + // min, max, and average of observed values + double min = 4; + double max = 5; + double avg = 6; +} + +message RateValue { + google.protobuf.Timestamp time = 1; + uint32 nonzero_count = 2; + uint32 total_count = 3; +} + +message TrendHdrValue { + google.protobuf.Timestamp time = 1; + + // histogram parameter - value multiplier aka smallest value, default = 1.0 + double min_resolution = 2; + // histogram parameter - number of significant digits used to calculate buckets formula, + // default = 2 + uint32 significant_digits = 3; + + // buckets + repeated uint32 counters = 4; + // index of first bucket in `counters` + uint32 lower_counter_index = 5; + // sum of observations + double sum = 6; + // count of observations + uint32 count = 7; + // smallest and largest observed value + double min_value = 8; + double max_value = 9; + + // counters for zero- and infinity-buckets + optional uint32 extra_low_values_counter = 10; + optional uint32 extra_high_values_counter = 11; +} diff --git a/output/cloud/output.go b/output/cloud/output.go index 16658eae9eb5..6be6b2acf6b3 100644 --- a/output/cloud/output.go +++ b/output/cloud/output.go @@ -17,6 +17,7 @@ import ( "go.k6.io/k6/errext" "go.k6.io/k6/errext/exitcodes" "go.k6.io/k6/output" + "go.k6.io/k6/output/cloud/expv2" "go.k6.io/k6/lib" "go.k6.io/k6/lib/consts" @@ -45,6 +46,8 @@ type Output struct { logger logrus.FieldLogger opts lib.Options + outv2 *expv2.Output + // TODO: optimize this // // Since the real-time metrics refactoring (https://github.com/k6io/k6/pull/678), @@ -163,6 +166,7 @@ func validateRequiredSystemTags(scriptTags *metrics.SystemTagSet) error { // Start calls the k6 Cloud API to initialize the test run, and then starts the // goroutine that would listen for metric samples and send them to the cloud. func (out *Output) Start() error { + // TODO: should we care about this in v2? if out.config.PushRefID.Valid { out.referenceID = out.config.PushRefID.String out.logger.WithField("referenceId", out.referenceID).Debug("directly pushing metrics without init") @@ -200,13 +204,28 @@ func (out *Output) Start() error { out.config = out.config.Apply(*response.ConfigOverride) } - out.startBackgroundProcesses() + // TODO: unit test this config override logic + if !out.config.UseVersion2.Bool { + // fallback on v1 + out.startBackgroundProcesses() + } else { + out.outv2, err = expv2.New(out.logger, out.config) + if err != nil { + return fmt.Errorf("failed to init the cloud output v2: %w", err) + } + out.outv2.SetReferenceID(out.referenceID) + err = out.outv2.Start() + if err != nil { + return fmt.Errorf("failed to startup the cloud output v2: %w", err) + } + } out.logger.WithFields(logrus.Fields{ "name": out.config.Name, "projectId": out.config.ProjectID, "duration": out.duration, "referenceId": out.referenceID, + "outputv2": out.config.UseVersion2, }).Debug("Started!") return nil } @@ -271,13 +290,21 @@ func (out *Output) Stop() error { // all metric samples are emitted, it makes a cloud API call to finish the test // run. If testErr was specified, it extracts the RunStatus from it. func (out *Output) StopWithTestError(testErr error) error { - out.logger.Debug("Stopping the cloud output...") - close(out.stopAggregation) - out.aggregationDone.Wait() // could be a no-op, if we have never started the aggregation - out.logger.Debug("Aggregation stopped, stopping metric emission...") - close(out.stopOutput) - out.outputDone.Wait() - out.logger.Debug("Metric emission stopped, calling cloud API...") + if out.outv2 == nil { + out.logger.Debug("Stopping the cloud output...") + close(out.stopAggregation) + out.aggregationDone.Wait() // could be a no-op, if we have never started the aggregation + out.logger.Debug("Aggregation stopped, stopping metric emission...") + close(out.stopOutput) + out.outputDone.Wait() + out.logger.Debug("Metric emission stopped, calling cloud API...") + } else { + err := out.outv2.StopWithTestError(testErr) + if err != nil { + out.logger.WithError(err).Warn("Failed to stop the cloud output v2") + } + } + err := out.testFinished(testErr) if err != nil { out.logger.WithFields(logrus.Fields{"error": err}).Warn("Failed to send test finished to the cloud") @@ -359,6 +386,11 @@ func (out *Output) SetTestRunStopCallback(stopFunc func(error)) { // called concurrently, so it defers as much of the work as possible to the // asynchronous goroutines initialized in Start(). func (out *Output) AddMetricSamples(sampleContainers []metrics.SampleContainer) { + if out.outv2 != nil { + out.outv2.AddMetricSamples(sampleContainers) + return + } + select { case <-out.stopSendingMetrics: return