From 7ba1898edbd4b226e706c6b9416253b48b5be4b9 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Thu, 16 Nov 2023 13:50:57 +0300 Subject: [PATCH 001/112] feat(chstorage): implement metrics storage --- internal/chstorage/columns_metrics.go | 51 ++ .../{columns.go => columns_traces.go} | 0 internal/chstorage/querier.go | 460 ----------------- internal/chstorage/querier_metrics.go | 359 ++++++++++++++ internal/chstorage/querier_traces.go | 466 ++++++++++++++++++ internal/chstorage/schema.go | 111 +---- internal/chstorage/schema_metrics.go | 61 +++ internal/chstorage/schema_traces.go | 92 ++++ 8 files changed, 1033 insertions(+), 567 deletions(-) create mode 100644 internal/chstorage/columns_metrics.go rename internal/chstorage/{columns.go => columns_traces.go} (100%) create mode 100644 internal/chstorage/querier_metrics.go create mode 100644 internal/chstorage/querier_traces.go create mode 100644 internal/chstorage/schema_metrics.go create mode 100644 internal/chstorage/schema_traces.go diff --git a/internal/chstorage/columns_metrics.go b/internal/chstorage/columns_metrics.go new file mode 100644 index 00000000..c6d14310 --- /dev/null +++ b/internal/chstorage/columns_metrics.go @@ -0,0 +1,51 @@ +package chstorage + +import ( + "github.com/ClickHouse/ch-go/proto" +) + +type metricColumns struct { + name *proto.ColLowCardinality[string] + ts *proto.ColDateTime64 + value proto.ColFloat64 + attributes proto.ColStr + resource proto.ColStr +} + +func newMetricColumns() *metricColumns { + return &metricColumns{ + name: new(proto.ColStr).LowCardinality(), + ts: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + } +} + +func (c *metricColumns) StaticColumns() []string { + return []string{ + "name", + "ts", + "value", + "attributes", + "resource", + } +} + +func (c *metricColumns) Input() proto.Input { + input := proto.Input{ + {Name: "name", Data: c.name}, + {Name: "ts", Data: c.ts}, + {Name: "value", Data: c.value}, + {Name: "attributes", Data: c.attributes}, + {Name: "resource", Data: c.resource}, + } + return input +} + +func (c *metricColumns) Result() proto.Results { + return proto.Results{ + {Name: "name", Data: c.name}, + {Name: "ts", Data: c.ts}, + {Name: "value", Data: &c.value}, + {Name: "attributes", Data: &c.attributes}, + {Name: "resource", Data: &c.resource}, + } +} diff --git a/internal/chstorage/columns.go b/internal/chstorage/columns_traces.go similarity index 100% rename from internal/chstorage/columns.go rename to internal/chstorage/columns_traces.go diff --git a/internal/chstorage/querier.go b/internal/chstorage/querier.go index fae4418d..a5aff8e6 100644 --- a/internal/chstorage/querier.go +++ b/internal/chstorage/querier.go @@ -1,25 +1,11 @@ package chstorage import ( - "context" - "fmt" - "strconv" - "strings" - - "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/chpool" - "github.com/ClickHouse/ch-go/proto" "go.opentelemetry.io/otel" - "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/trace" - "github.com/go-faster/errors" - - "github.com/go-faster/oteldb/internal/iterators" - "github.com/go-faster/oteldb/internal/otelstorage" - "github.com/go-faster/oteldb/internal/traceql" - "github.com/go-faster/oteldb/internal/traceql/traceqlengine" "github.com/go-faster/oteldb/internal/tracestorage" ) @@ -64,449 +50,3 @@ func NewQuerier(c *chpool.Pool, opts QuerierOptions) (*Querier, error) { tracer: otel.Tracer("chstorage.Querier"), }, nil } - -// SearchTags performs search by given tags. -func (q *Querier) SearchTags(ctx context.Context, tags map[string]string, opts tracestorage.SearchTagsOptions) (_ iterators.Iterator[tracestorage.Span], rerr error) { - table := q.tables.Spans - - ctx, span := q.tracer.Start(ctx, "TagNames", - trace.WithAttributes( - attribute.Int("chstorage.tags_count", len(tags)), - attribute.Int64("chstorage.start_range", int64(opts.Start)), - attribute.Int64("chstorage.end_range", int64(opts.End)), - attribute.Int64("chstorage.max_duration", int64(opts.MaxDuration)), - attribute.Int64("chstorage.min_duration", int64(opts.MinDuration)), - attribute.String("chstorage.table", table), - ), - ) - defer func() { - if rerr != nil { - span.RecordError(rerr) - } - span.End() - }() - - var query strings.Builder - fmt.Fprintf(&query, `SELECT * FROM %#[1]q WHERE trace_id IN ( - SELECT trace_id FROM %#[1]q WHERE true - `, table) - for key, value := range tags { - if key == "name" { - fmt.Fprintf(&query, " AND name = %s", singleQuoted(value)) - continue - } - - query.WriteString(" AND (") - for i, prefix := range []string{ - "attrs", - "scope_attrs", - "resource_attrs", - } { - if i != 0 { - query.WriteString(" OR ") - } - for i, column := range []string{ - "str", - "int", - "float", - "bool", - "bytes", - } { - if i != 0 { - query.WriteString(" OR ") - } - fmt.Fprintf(&query, - `toString( %[1]s_%[2]s_values[indexOf(%[1]s_%[2]s_keys, %[3]s)] ) = %[4]s`, - prefix, column, singleQuoted(key), singleQuoted(value), - ) - } - query.WriteByte('\n') - } - query.WriteByte(')') - } - query.WriteByte(')') - - if s := opts.Start; s != 0 { - fmt.Fprintf(&query, " AND toUnixTimestamp64Nano(start) >= %d", s) - } - if e := opts.End; e != 0 { - fmt.Fprintf(&query, " AND toUnixTimestamp64Nano(end) <= %d", e) - } - if d := opts.MinDuration; d != 0 { - fmt.Fprintf(&query, " AND (toUnixTimestamp64Nano(end) - toUnixTimestamp64Nano(start)) >= %d", d) - } - if d := opts.MaxDuration; d != 0 { - fmt.Fprintf(&query, " AND (toUnixTimestamp64Nano(end) - toUnixTimestamp64Nano(start)) <= %d", d) - } - return q.querySpans(ctx, query.String()) -} - -// TagNames returns all available tag names. -func (q *Querier) TagNames(ctx context.Context) (r []string, rerr error) { - table := q.tables.Tags - - ctx, span := q.tracer.Start(ctx, "TagNames", - trace.WithAttributes( - attribute.String("chstorage.table", table), - ), - ) - defer func() { - if rerr != nil { - span.RecordError(rerr) - } - span.End() - }() - - data := new(proto.ColStr).LowCardinality() - if err := q.ch.Do(ctx, ch.Query{ - Body: fmt.Sprintf("SELECT DISTINCT name FROM %#q", table), - Result: proto.ResultColumn{ - Name: "name", - Data: data, - }, - OnResult: func(ctx context.Context, block proto.Block) error { - r = append(r, data.Values...) - return nil - }, - }); err != nil { - return nil, errors.Wrap(err, "query") - } - return r, nil -} - -// TagValues returns all available tag values for given tag. -func (q *Querier) TagValues(ctx context.Context, tagName string) (_ iterators.Iterator[tracestorage.Tag], rerr error) { - table := q.tables.Tags - - ctx, span := q.tracer.Start(ctx, "TagValues", - trace.WithAttributes( - attribute.String("chstorage.tag_to_query", tagName), - attribute.String("chstorage.table", table), - ), - ) - defer func() { - if rerr != nil { - span.RecordError(rerr) - } - span.End() - }() - - var ( - value proto.ColStr - valueType proto.ColEnum8 - - r []tracestorage.Tag - ) - - if err := q.ch.Do(ctx, ch.Query{ - Body: fmt.Sprintf("SELECT DISTINCT value, value_type FROM %#q WHERE name = %s", table, singleQuoted(tagName)), - Result: proto.Results{ - {Name: "value", Data: &value}, - {Name: "value_type", Data: proto.Wrap(&valueType, valueTypeDDL)}, - }, - OnResult: func(ctx context.Context, block proto.Block) error { - return value.ForEach(func(i int, value string) error { - typ := valueType.Row(i) - r = append(r, tracestorage.Tag{ - Name: tagName, - Value: value, - Type: int32(typ), - }) - return nil - }) - }, - }); err != nil { - return nil, errors.Wrap(err, "query") - } - - return iterators.Slice(r), nil -} - -// TraceByID returns spans of given trace. -func (q *Querier) TraceByID(ctx context.Context, id otelstorage.TraceID, opts tracestorage.TraceByIDOptions) (_ iterators.Iterator[tracestorage.Span], rerr error) { - table := q.tables.Spans - - ctx, span := q.tracer.Start(ctx, "TraceByID", - trace.WithAttributes( - attribute.String("chstorage.id_to_query", id.Hex()), - attribute.Int64("chstorage.start_range", int64(opts.Start)), - attribute.Int64("chstorage.end_range", int64(opts.End)), - attribute.String("chstorage.table", table), - ), - ) - defer func() { - if rerr != nil { - span.RecordError(rerr) - } - span.End() - }() - - query := fmt.Sprintf("SELECT * FROM %#q WHERE trace_id = %s", table, singleQuoted(id.Hex())) - if s := opts.Start; s != 0 { - query += fmt.Sprintf(" AND toUnixTimestamp64Nano(start) >= %d", s) - } - if e := opts.End; e != 0 { - query += fmt.Sprintf(" AND toUnixTimestamp64Nano(end) <= %d", e) - } - return q.querySpans(ctx, query) -} - -var _ traceqlengine.Querier = (*Querier)(nil) - -// SelectSpansets get spansets from storage. -func (q *Querier) SelectSpansets(ctx context.Context, params traceqlengine.SelectSpansetsParams) (_ iterators.Iterator[traceqlengine.Trace], rerr error) { - ctx, span := q.tracer.Start(ctx, "SelectSpansets", - trace.WithAttributes( - attribute.String("chstorage.span_matcher_operation", params.Op.String()), - attribute.Int("chstorage.span_matchers", len(params.Matchers)), - attribute.Int64("chstorage.start_range", int64(params.Start)), - attribute.Int64("chstorage.end_range", int64(params.End)), - attribute.Int64("chstorage.max_duration", int64(params.MaxDuration)), - attribute.Int64("chstorage.min_duration", int64(params.MinDuration)), - attribute.Int("chstorage.limit", params.Limit), - ), - ) - defer func() { - if rerr != nil { - span.RecordError(rerr) - } - span.End() - }() - - query := q.buildSpansetsQuery(span, params) - - iter, err := q.querySpans(ctx, query) - if err != nil { - return nil, errors.Wrap(err, "query traces") - } - defer func() { - _ = iter.Close() - }() - - var ( - traces = map[otelstorage.TraceID][]tracestorage.Span{} - val tracestorage.Span - ) - for iter.Next(&val) { - traces[val.TraceID] = append(traces[val.TraceID], val) - } - if err := iter.Err(); err != nil { - return nil, err - } - - var ( - result = make([]traceqlengine.Trace, 0, len(traces)) - spansCount int - ) - for id, spans := range traces { - spansCount += len(spans) - result = append(result, traceqlengine.Trace{ - TraceID: id, - Spans: spans, - }) - } - span.SetAttributes( - attribute.Int("chstorage.queried_spans", spansCount), - attribute.Int("chstorage.queried_traces", len(result)), - ) - - return iterators.Slice(result), nil -} - -func (q *Querier) buildSpansetsQuery(span trace.Span, params traceqlengine.SelectSpansetsParams) string { - var ( - query strings.Builder - table = q.tables.Spans - ) - - fmt.Fprintf(&query, `SELECT * FROM %#[1]q WHERE trace_id IN ( - SELECT trace_id FROM %#[1]q WHERE true - `, table) - - var ( - dropped int - writeNext = func() { - if params.Op == traceql.SpansetOpAnd { - query.WriteString("\nAND ") - } else { - query.WriteString("\nOR ") - } - } - ) - for _, matcher := range params.Matchers { - if matcher.Op == 0 { - writeNext() - - // Just query spans with this attribute. - attr := matcher.Attribute - query.WriteString("(\n") - for i, prefix := range getAttributeColumns(attr) { - if i != 0 { - query.WriteString(" OR ") - } - for i, column := range []string{ - "str", - "int", - "float", - "bool", - "bytes", - } { - if i != 0 { - query.WriteString(" OR ") - } - fmt.Fprintf(&query, - `has(%s_%s_keys, %s)`, - prefix, column, singleQuoted(attr.Name), - ) - } - query.WriteByte('\n') - } - query.WriteString("\n)") - } - - var cmp string - switch matcher.Op { - case traceql.OpEq: - cmp = "=" - case traceql.OpNotEq: - cmp = "!=" - case traceql.OpGt: - cmp = ">" - case traceql.OpGte: - cmp = ">=" - case traceql.OpLt: - cmp = "<" - case traceql.OpLte: - cmp = "<=" - case traceql.OpRe: - cmp = "REGEXP" - default: - // Unsupported for now. - dropped++ - continue - } - - var value, typeSuffix string - switch s := matcher.Static; s.Type { - case traceql.TypeString: - value = singleQuoted(s.Str) - typeSuffix = "str" - case traceql.TypeInt: - value = strconv.FormatInt(s.AsInt(), 10) - typeSuffix = "int" - case traceql.TypeNumber: - value = strconv.FormatFloat(s.AsNumber(), 'f', -1, 64) - typeSuffix = "float" - case traceql.TypeBool: - if s.AsBool() { - value = "true" - } else { - value = "false" - } - typeSuffix = "bool" - case traceql.TypeDuration: - value = strconv.FormatInt(s.AsDuration().Nanoseconds(), 10) - typeSuffix = "int" - case traceql.TypeSpanStatus: - value = strconv.Itoa(int(s.AsSpanStatus())) - typeSuffix = "int" - case traceql.TypeSpanKind: - value = strconv.Itoa(int(s.AsSpanKind())) - typeSuffix = "int" - default: - // Unsupported for now. - dropped++ - continue - } - - writeNext() - switch attr := matcher.Attribute; attr.Prop { - case traceql.SpanDuration: - fmt.Fprintf(&query, "(toUnixTimestamp64Nano(end)-toUnixTimestamp64Nano(start)) %s %s", cmp, value) - case traceql.SpanName: - fmt.Fprintf(&query, "name %s %s", cmp, value) - case traceql.SpanStatus: - fmt.Fprintf(&query, "status_code %s %s", cmp, value) - case traceql.SpanKind: - fmt.Fprintf(&query, "kind %s %s", cmp, value) - case traceql.SpanParent, - traceql.SpanChildCount, - traceql.RootSpanName, - traceql.RootServiceName, - traceql.TraceDuration: - // Unsupported yet. - dropped++ - query.WriteString("true") - default: - // SpanAttribute - query.WriteString("(\n") - for i, column := range getAttributeColumns(attr) { - if i != 0 { - query.WriteString("\nOR ") - } - fmt.Fprintf(&query, "%[1]s_%[2]s_values[indexOf(%[1]s_%[2]s_keys, %[3]s)] %[4]s %[5]s", - column, typeSuffix, - singleQuoted(attr.Name), - cmp, value, - ) - } - query.WriteString("\n)") - } - } - query.WriteString("\n)") - if s := params.Start; s != 0 { - fmt.Fprintf(&query, " AND toUnixTimestamp64Nano(start) >= %d", s) - } - if e := params.End; e != 0 { - fmt.Fprintf(&query, " AND toUnixTimestamp64Nano(end) <= %d", e) - } - - span.SetAttributes( - attribute.Int("chstorage.unsupported_span_matchers", dropped), - attribute.String("chstorage.table", table), - ) - return query.String() -} - -func getAttributeColumns(attr traceql.Attribute) []string { - if attr.Prop != traceql.SpanAttribute || attr.Parent { - return nil - } - switch attr.Scope { - case traceql.ScopeNone: - return []string{ - "attrs", - "scope_attrs", - "resource_attrs", - } - case traceql.ScopeResource: - return []string{ - "scope_attrs", - "resource_attrs", - } - case traceql.ScopeSpan: - return []string{ - "attrs", - } - default: - return nil - } -} - -func (q *Querier) querySpans(ctx context.Context, query string) (iterators.Iterator[tracestorage.Span], error) { - c := newSpanColumns() - - var r []tracestorage.Span - if err := q.ch.Do(ctx, ch.Query{ - Body: query, - Result: c.Result(), - OnResult: func(ctx context.Context, block proto.Block) error { - r = c.ReadRowsTo(r) - return nil - }, - }); err != nil { - return nil, errors.Wrap(err, "query") - } - - return iterators.Slice(r), nil -} diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go new file mode 100644 index 00000000..daf3c108 --- /dev/null +++ b/internal/chstorage/querier_metrics.go @@ -0,0 +1,359 @@ +package chstorage + +import ( + "context" + "fmt" + "slices" + "strings" + "time" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/errors" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/util/annotations" +) + +var _ storage.Queryable = (*Querier)(nil) + +// Querier returns a new Querier on the storage. +func (q *Querier) Querier(mint, maxt int64) (storage.Querier, error) { + var minTime, maxTime time.Time + if mint > 0 { + minTime = time.UnixMilli(mint) + } + if maxt > 0 { + maxTime = time.UnixMilli(maxt) + } + return &promQuerier{ + ch: q.ch, + mint: minTime, + maxt: maxTime, + }, nil +} + +type promQuerier struct { + ch chClient + table Tables + + mint time.Time + maxt time.Time +} + +var _ storage.Querier = (*promQuerier)(nil) + +// LabelValues returns all potential values for a label name. +// It is not safe to use the strings beyond the lifetime of the querier. +// If matchers are specified the returned result set is reduced +// to label values of metrics matching the matchers. +func (p *promQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + // TODO: Implement + return nil, nil, nil +} + +// LabelNames returns all the unique label names present in the block in sorted order. +// If matchers are specified the returned result set is reduced +// to label names of metrics matching the matchers. +func (p *promQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + // TODO: Implement + return nil, nil, nil +} + +// Close releases the resources of the Querier. +func (p *promQuerier) Close() error { + return nil +} + +// Select returns a set of series that matches the given label matchers. +// Caller can specify if it requires returned series to be sorted. Prefer not requiring sorting for better performance. +// It allows passing hints that can help in optimising select, but it's up to implementation how this is used if used at all. +func (p *promQuerier) Select(ctx context.Context, _ bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { + ss, err := p.selectSeries(ctx, hints, matchers...) + if err != nil { + return storage.ErrSeriesSet(err) + } + return ss +} + +type seriesKey struct { + attributes string + resource string +} + +func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHints, matchers ...*labels.Matcher) (storage.SeriesSet, error) { + var ( + start = p.mint + end = p.maxt + ) + if hints != nil { + if t := time.UnixMilli(hints.Start); t.After(start) { + start = t + } + if t := time.UnixMilli(hints.End); t.Before(end) { + end = t + } + } + + var query strings.Builder + fmt.Fprintf(&query, "SELECT * FROM %#[1]q WHERE true\n", p.table.Points) + if !start.IsZero() { + fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(ts) >= %d\n", start.UnixNano()) + } + if !end.IsZero() { + fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(ts) <= %d\n", end.UnixNano()) + } + for _, m := range matchers { + query.WriteString("\t(") + + switch m.Type { + case labels.MatchEqual, labels.MatchRegexp: + query.WriteString("AND ") + case labels.MatchNotEqual, labels.MatchNotRegexp: + query.WriteString("AND NOT ") + default: + return nil, errors.Errorf("unexpected type %q", m.Type) + } + + { + selectors := []string{ + "name", + } + if m.Name != "__name__" { + selectors = []string{ + fmt.Sprintf("JSONExtractString(attributes, %s)", singleQuoted(m.Name)), + fmt.Sprintf("JSONExtractString(resource, %s)", singleQuoted(m.Name)), + } + } + query.WriteString("\t\t(") + for i, sel := range selectors { + if i != 0 { + query.WriteString("\t\tOR") + } + // Note: predicate negated above. + switch m.Type { + case labels.MatchEqual, labels.MatchNotEqual: + fmt.Fprintf(&query, "%s = %s\n", sel, singleQuoted(m.Value)) + case labels.MatchRegexp, labels.MatchNotRegexp: + fmt.Fprintf(&query, "%s REGEXP %s\n", sel, singleQuoted(m.Value)) + default: + return nil, errors.Errorf("unexpected type %q", m.Type) + } + } + query.WriteString("\t)") + } + + query.WriteString(")\n") + } + query.WriteString("ORDER BY ts") + + return p.doQuery(ctx, query.String()) +} + +func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.SeriesSet, error) { + type seriesWithLabels struct { + series *series + labels map[string]string + } + + var ( + set = map[seriesKey]seriesWithLabels{} + c = newMetricColumns() + ) + if err := p.ch.Do(ctx, ch.Query{ + Body: query, + Result: c.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + for i := 0; i < c.ts.Rows(); i++ { + value := c.value.Row(i) + ts := c.ts.Row(i) + attributes := c.attributes.Row(i) + resource := c.resource.Row(i) + + key := seriesKey{ + attributes: attributes, + resource: resource, + } + s, ok := set[key] + if !ok { + s = seriesWithLabels{ + series: &series{}, + labels: map[string]string{}, + } + set[key] = s + } + + s.series.values = append(s.series.values, value) + s.series.ts = append(s.series.ts, ts.UnixMilli()) + if err := parseLabels(resource, s.labels); err != nil { + return errors.Wrap(err, "parse resource") + } + if err := parseLabels(attributes, s.labels); err != nil { + return errors.Wrap(err, "parse attributes") + } + } + return nil + }, + }); err != nil { + return nil, errors.Wrap(err, "do query") + } + + var ( + result = make([]*series, 0, len(set)) + lb labels.ScratchBuilder + ) + for _, s := range set { + lb.Reset() + for key, value := range s.labels { + lb.Add(key, value) + } + lb.Sort() + s.series.labels = lb.Labels() + result = append(result, s.series) + } + + return &seriesSet{ + set: result, + }, nil +} + +type seriesSet struct { + set []*series + n int +} + +func newSeriesSet(set []*series) *seriesSet { + return &seriesSet{ + set: set, + n: -1, + } +} + +var _ storage.SeriesSet = (*seriesSet)(nil) + +func (s *seriesSet) Next() bool { + if s.n+1 >= len(s.set) { + return false + } + s.n++ + return true +} + +// At returns full series. Returned series should be iterable even after Next is called. +func (s *seriesSet) At() storage.Series { + return s.set[s.n] +} + +// The error that iteration as failed with. +// When an error occurs, set cannot continue to iterate. +func (s *seriesSet) Err() error { + return nil +} + +// A collection of warnings for the whole set. +// Warnings could be return even iteration has not failed with error. +func (s *seriesSet) Warnings() annotations.Annotations { + return nil +} + +type series struct { + labels labels.Labels + values []float64 + ts []int64 +} + +var _ storage.Series = (*series)(nil) + +// Labels returns the complete set of labels. For series it means all labels identifying the series. +func (s *series) Labels() labels.Labels { + return s.labels +} + +// Iterator returns an iterator of the data of the series. +// The iterator passed as argument is for re-use, if not nil. +// Depending on implementation, the iterator can +// be re-used or a new iterator can be allocated. +func (s *series) Iterator(chunkenc.Iterator) chunkenc.Iterator { + return newPointIterator(s.values, s.ts) +} + +type pointIterator struct { + values []float64 + ts []int64 + n int +} + +var _ chunkenc.Iterator = (*pointIterator)(nil) + +func newPointIterator(values []float64, ts []int64) *pointIterator { + return &pointIterator{ + values: values, + ts: ts, + n: -1, + } +} + +// Next advances the iterator by one and returns the type of the value +// at the new position (or ValNone if the iterator is exhausted). +func (p *pointIterator) Next() chunkenc.ValueType { + if p.n+1 >= len(p.values) { + return chunkenc.ValNone + } + p.n++ + return chunkenc.ValFloat +} + +// Seek advances the iterator forward to the first sample with a +// timestamp equal or greater than t. If the current sample found by a +// previous `Next` or `Seek` operation already has this property, Seek +// has no effect. If a sample has been found, Seek returns the type of +// its value. Otherwise, it returns ValNone, after which the iterator is +// exhausted. +func (p *pointIterator) Seek(seek int64) chunkenc.ValueType { + // Find the closest value. + idx, _ := slices.BinarySearch(p.ts, seek) + if idx >= len(p.ts) { + p.n = len(p.ts) + return chunkenc.ValNone + } + p.n = idx - 1 + return chunkenc.ValFloat +} + +// At returns the current timestamp/value pair if the value is a float. +// Before the iterator has advanced, the behavior is unspecified. +func (p *pointIterator) At() (t int64, v float64) { + t = p.AtT() + v = p.values[p.n] + return t, v +} + +// AtHistogram returns the current timestamp/value pair if the value is +// a histogram with integer counts. Before the iterator has advanced, +// the behavior is unspecified. +func (p *pointIterator) AtHistogram() (int64, *histogram.Histogram) { + return 0, nil +} + +// AtFloatHistogram returns the current timestamp/value pair if the +// value is a histogram with floating-point counts. It also works if the +// value is a histogram with integer counts, in which case a +// FloatHistogram copy of the histogram is returned. Before the iterator +// has advanced, the behavior is unspecified. +func (p *pointIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) { + return 0, nil +} + +// AtT returns the current timestamp. +// Before the iterator has advanced, the behavior is unspecified. +func (p *pointIterator) AtT() int64 { + return p.ts[p.n] +} + +// Err returns the current error. It should be used only after the +// iterator is exhausted, i.e. `Next` or `Seek` have returned ValNone. +func (p *pointIterator) Err() error { + return nil +} diff --git a/internal/chstorage/querier_traces.go b/internal/chstorage/querier_traces.go new file mode 100644 index 00000000..1346dc98 --- /dev/null +++ b/internal/chstorage/querier_traces.go @@ -0,0 +1,466 @@ +package chstorage + +import ( + "context" + "fmt" + "strconv" + "strings" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/errors" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/trace" + + "github.com/go-faster/oteldb/internal/iterators" + "github.com/go-faster/oteldb/internal/otelstorage" + "github.com/go-faster/oteldb/internal/traceql" + "github.com/go-faster/oteldb/internal/traceql/traceqlengine" + "github.com/go-faster/oteldb/internal/tracestorage" +) + +// SearchTags performs search by given tags. +func (q *Querier) SearchTags(ctx context.Context, tags map[string]string, opts tracestorage.SearchTagsOptions) (_ iterators.Iterator[tracestorage.Span], rerr error) { + table := q.tables.Spans + + ctx, span := q.tracer.Start(ctx, "TagNames", + trace.WithAttributes( + attribute.Int("chstorage.tags_count", len(tags)), + attribute.Int64("chstorage.start_range", int64(opts.Start)), + attribute.Int64("chstorage.end_range", int64(opts.End)), + attribute.Int64("chstorage.max_duration", int64(opts.MaxDuration)), + attribute.Int64("chstorage.min_duration", int64(opts.MinDuration)), + attribute.String("chstorage.table", table), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + var query strings.Builder + fmt.Fprintf(&query, `SELECT * FROM %#[1]q WHERE trace_id IN ( + SELECT trace_id FROM %#[1]q WHERE true + `, table) + for key, value := range tags { + if key == "name" { + fmt.Fprintf(&query, " AND name = %s", singleQuoted(value)) + continue + } + + query.WriteString(" AND (") + for i, prefix := range []string{ + "attrs", + "scope_attrs", + "resource_attrs", + } { + if i != 0 { + query.WriteString(" OR ") + } + for i, column := range []string{ + "str", + "int", + "float", + "bool", + "bytes", + } { + if i != 0 { + query.WriteString(" OR ") + } + fmt.Fprintf(&query, + `toString( %[1]s_%[2]s_values[indexOf(%[1]s_%[2]s_keys, %[3]s)] ) = %[4]s`, + prefix, column, singleQuoted(key), singleQuoted(value), + ) + } + query.WriteByte('\n') + } + query.WriteByte(')') + } + query.WriteByte(')') + + if s := opts.Start; s != 0 { + fmt.Fprintf(&query, " AND toUnixTimestamp64Nano(start) >= %d", s) + } + if e := opts.End; e != 0 { + fmt.Fprintf(&query, " AND toUnixTimestamp64Nano(end) <= %d", e) + } + if d := opts.MinDuration; d != 0 { + fmt.Fprintf(&query, " AND (toUnixTimestamp64Nano(end) - toUnixTimestamp64Nano(start)) >= %d", d) + } + if d := opts.MaxDuration; d != 0 { + fmt.Fprintf(&query, " AND (toUnixTimestamp64Nano(end) - toUnixTimestamp64Nano(start)) <= %d", d) + } + return q.querySpans(ctx, query.String()) +} + +// TagNames returns all available tag names. +func (q *Querier) TagNames(ctx context.Context) (r []string, rerr error) { + table := q.tables.Tags + + ctx, span := q.tracer.Start(ctx, "TagNames", + trace.WithAttributes( + attribute.String("chstorage.table", table), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + data := new(proto.ColStr).LowCardinality() + if err := q.ch.Do(ctx, ch.Query{ + Body: fmt.Sprintf("SELECT DISTINCT name FROM %#q", table), + Result: proto.ResultColumn{ + Name: "name", + Data: data, + }, + OnResult: func(ctx context.Context, block proto.Block) error { + r = append(r, data.Values...) + return nil + }, + }); err != nil { + return nil, errors.Wrap(err, "query") + } + return r, nil +} + +// TagValues returns all available tag values for given tag. +func (q *Querier) TagValues(ctx context.Context, tagName string) (_ iterators.Iterator[tracestorage.Tag], rerr error) { + table := q.tables.Tags + + ctx, span := q.tracer.Start(ctx, "TagValues", + trace.WithAttributes( + attribute.String("chstorage.tag_to_query", tagName), + attribute.String("chstorage.table", table), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + var ( + value proto.ColStr + valueType proto.ColEnum8 + + r []tracestorage.Tag + ) + + if err := q.ch.Do(ctx, ch.Query{ + Body: fmt.Sprintf("SELECT DISTINCT value, value_type FROM %#q WHERE name = %s", table, singleQuoted(tagName)), + Result: proto.Results{ + {Name: "value", Data: &value}, + {Name: "value_type", Data: proto.Wrap(&valueType, valueTypeDDL)}, + }, + OnResult: func(ctx context.Context, block proto.Block) error { + return value.ForEach(func(i int, value string) error { + typ := valueType.Row(i) + r = append(r, tracestorage.Tag{ + Name: tagName, + Value: value, + Type: int32(typ), + }) + return nil + }) + }, + }); err != nil { + return nil, errors.Wrap(err, "query") + } + + return iterators.Slice(r), nil +} + +// TraceByID returns spans of given trace. +func (q *Querier) TraceByID(ctx context.Context, id otelstorage.TraceID, opts tracestorage.TraceByIDOptions) (_ iterators.Iterator[tracestorage.Span], rerr error) { + table := q.tables.Spans + + ctx, span := q.tracer.Start(ctx, "TraceByID", + trace.WithAttributes( + attribute.String("chstorage.id_to_query", id.Hex()), + attribute.Int64("chstorage.start_range", int64(opts.Start)), + attribute.Int64("chstorage.end_range", int64(opts.End)), + attribute.String("chstorage.table", table), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + query := fmt.Sprintf("SELECT * FROM %#q WHERE trace_id = %s", table, singleQuoted(id.Hex())) + if s := opts.Start; s != 0 { + query += fmt.Sprintf(" AND toUnixTimestamp64Nano(start) >= %d", s) + } + if e := opts.End; e != 0 { + query += fmt.Sprintf(" AND toUnixTimestamp64Nano(end) <= %d", e) + } + return q.querySpans(ctx, query) +} + +var _ traceqlengine.Querier = (*Querier)(nil) + +// SelectSpansets get spansets from storage. +func (q *Querier) SelectSpansets(ctx context.Context, params traceqlengine.SelectSpansetsParams) (_ iterators.Iterator[traceqlengine.Trace], rerr error) { + ctx, span := q.tracer.Start(ctx, "SelectSpansets", + trace.WithAttributes( + attribute.String("chstorage.span_matcher_operation", params.Op.String()), + attribute.Int("chstorage.span_matchers", len(params.Matchers)), + attribute.Int64("chstorage.start_range", int64(params.Start)), + attribute.Int64("chstorage.end_range", int64(params.End)), + attribute.Int64("chstorage.max_duration", int64(params.MaxDuration)), + attribute.Int64("chstorage.min_duration", int64(params.MinDuration)), + attribute.Int("chstorage.limit", params.Limit), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + query := q.buildSpansetsQuery(span, params) + + iter, err := q.querySpans(ctx, query) + if err != nil { + return nil, errors.Wrap(err, "query traces") + } + defer func() { + _ = iter.Close() + }() + + var ( + traces = map[otelstorage.TraceID][]tracestorage.Span{} + val tracestorage.Span + ) + for iter.Next(&val) { + traces[val.TraceID] = append(traces[val.TraceID], val) + } + if err := iter.Err(); err != nil { + return nil, err + } + + var ( + result = make([]traceqlengine.Trace, 0, len(traces)) + spansCount int + ) + for id, spans := range traces { + spansCount += len(spans) + result = append(result, traceqlengine.Trace{ + TraceID: id, + Spans: spans, + }) + } + span.SetAttributes( + attribute.Int("chstorage.queried_spans", spansCount), + attribute.Int("chstorage.queried_traces", len(result)), + ) + + return iterators.Slice(result), nil +} + +func (q *Querier) buildSpansetsQuery(span trace.Span, params traceqlengine.SelectSpansetsParams) string { + var ( + query strings.Builder + table = q.tables.Spans + ) + + fmt.Fprintf(&query, `SELECT * FROM %#[1]q WHERE trace_id IN ( + SELECT trace_id FROM %#[1]q WHERE true + `, table) + + var ( + dropped int + writeNext = func() { + if params.Op == traceql.SpansetOpAnd { + query.WriteString("\nAND ") + } else { + query.WriteString("\nOR ") + } + } + ) + for _, matcher := range params.Matchers { + if matcher.Op == 0 { + writeNext() + + // Just query spans with this attribute. + attr := matcher.Attribute + query.WriteString("(\n") + for i, prefix := range getTraceQLAttributeColumns(attr) { + if i != 0 { + query.WriteString(" OR ") + } + for i, column := range []string{ + "str", + "int", + "float", + "bool", + "bytes", + } { + if i != 0 { + query.WriteString(" OR ") + } + fmt.Fprintf(&query, + `has(%s_%s_keys, %s)`, + prefix, column, singleQuoted(attr.Name), + ) + } + query.WriteByte('\n') + } + query.WriteString("\n)") + } + + var cmp string + switch matcher.Op { + case traceql.OpEq: + cmp = "=" + case traceql.OpNotEq: + cmp = "!=" + case traceql.OpGt: + cmp = ">" + case traceql.OpGte: + cmp = ">=" + case traceql.OpLt: + cmp = "<" + case traceql.OpLte: + cmp = "<=" + case traceql.OpRe: + cmp = "REGEXP" + default: + // Unsupported for now. + dropped++ + continue + } + + var value, typeSuffix string + switch s := matcher.Static; s.Type { + case traceql.TypeString: + value = singleQuoted(s.Str) + typeSuffix = "str" + case traceql.TypeInt: + value = strconv.FormatInt(s.AsInt(), 10) + typeSuffix = "int" + case traceql.TypeNumber: + value = strconv.FormatFloat(s.AsNumber(), 'f', -1, 64) + typeSuffix = "float" + case traceql.TypeBool: + if s.AsBool() { + value = "true" + } else { + value = "false" + } + typeSuffix = "bool" + case traceql.TypeDuration: + value = strconv.FormatInt(s.AsDuration().Nanoseconds(), 10) + typeSuffix = "int" + case traceql.TypeSpanStatus: + value = strconv.Itoa(int(s.AsSpanStatus())) + typeSuffix = "int" + case traceql.TypeSpanKind: + value = strconv.Itoa(int(s.AsSpanKind())) + typeSuffix = "int" + default: + // Unsupported for now. + dropped++ + continue + } + + writeNext() + switch attr := matcher.Attribute; attr.Prop { + case traceql.SpanDuration: + fmt.Fprintf(&query, "(toUnixTimestamp64Nano(end)-toUnixTimestamp64Nano(start)) %s %s", cmp, value) + case traceql.SpanName: + fmt.Fprintf(&query, "name %s %s", cmp, value) + case traceql.SpanStatus: + fmt.Fprintf(&query, "status_code %s %s", cmp, value) + case traceql.SpanKind: + fmt.Fprintf(&query, "kind %s %s", cmp, value) + case traceql.SpanParent, + traceql.SpanChildCount, + traceql.RootSpanName, + traceql.RootServiceName, + traceql.TraceDuration: + // Unsupported yet. + dropped++ + query.WriteString("true") + default: + // SpanAttribute + query.WriteString("(\n") + for i, column := range getTraceQLAttributeColumns(attr) { + if i != 0 { + query.WriteString("\nOR ") + } + fmt.Fprintf(&query, "%[1]s_%[2]s_values[indexOf(%[1]s_%[2]s_keys, %[3]s)] %[4]s %[5]s", + column, typeSuffix, + singleQuoted(attr.Name), + cmp, value, + ) + } + query.WriteString("\n)") + } + } + query.WriteString("\n)") + if s := params.Start; s != 0 { + fmt.Fprintf(&query, " AND toUnixTimestamp64Nano(start) >= %d", s) + } + if e := params.End; e != 0 { + fmt.Fprintf(&query, " AND toUnixTimestamp64Nano(end) <= %d", e) + } + + span.SetAttributes( + attribute.Int("chstorage.unsupported_span_matchers", dropped), + attribute.String("chstorage.table", table), + ) + return query.String() +} + +func getTraceQLAttributeColumns(attr traceql.Attribute) []string { + if attr.Prop != traceql.SpanAttribute || attr.Parent { + return nil + } + switch attr.Scope { + case traceql.ScopeNone: + return []string{ + "attrs", + "scope_attrs", + "resource_attrs", + } + case traceql.ScopeResource: + return []string{ + "scope_attrs", + "resource_attrs", + } + case traceql.ScopeSpan: + return []string{ + "attrs", + } + default: + return nil + } +} + +func (q *Querier) querySpans(ctx context.Context, query string) (iterators.Iterator[tracestorage.Span], error) { + c := newSpanColumns() + + var r []tracestorage.Span + if err := q.ch.Do(ctx, ch.Query{ + Body: query, + Result: c.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + r = c.ReadRowsTo(r) + return nil + }, + }); err != nil { + return nil, errors.Wrap(err, "query") + } + + return iterators.Slice(r), nil +} diff --git a/internal/chstorage/schema.go b/internal/chstorage/schema.go index 6dc3b1e1..5b0ab708 100644 --- a/internal/chstorage/schema.go +++ b/internal/chstorage/schema.go @@ -8,122 +8,19 @@ import ( "github.com/go-faster/errors" ) -const ( - spansSchema = `CREATE TABLE IF NOT EXISTS %s -( - trace_id UUID, - span_id UInt64, - trace_state String, - parent_span_id UInt64, - name LowCardinality(String), - kind Enum8( - 'KIND_UNSPECIFIED' = 0, - 'KIND_INTERNAL' = 1, - 'KIND_SERVER' = 2, - 'KIND_CLIENT' = 3, - 'KIND_PRODUCER' = 4, - 'KIND_CONSUMER' = 5 - ), - start DateTime64(9), - end DateTime64(9), - attrs_str_keys Array(LowCardinality(String)), - attrs_str_values Array(String), - attrs_int_keys Array(LowCardinality(String)), - attrs_int_values Array(Int64), - attrs_float_keys Array(LowCardinality(String)), - attrs_float_values Array(Float64), - attrs_bool_keys Array(LowCardinality(String)), - attrs_bool_values Array(Bool), - attrs_bytes_keys Array(LowCardinality(String)), - attrs_bytes_values Array(String), - status_code Int32, - status_message String, - - batch_id UUID, - resource_attrs_str_keys Array(LowCardinality(String)), - resource_attrs_str_values Array(String), - resource_attrs_int_keys Array(LowCardinality(String)), - resource_attrs_int_values Array(Int64), - resource_attrs_float_keys Array(LowCardinality(String)), - resource_attrs_float_values Array(Float64), - resource_attrs_bool_keys Array(LowCardinality(String)), - resource_attrs_bool_values Array(Bool), - resource_attrs_bytes_keys Array(LowCardinality(String)), - resource_attrs_bytes_values Array(String), - - scope_name String, - scope_version String, - scope_attrs_str_keys Array(LowCardinality(String)), - scope_attrs_str_values Array(String), - scope_attrs_int_keys Array(LowCardinality(String)), - scope_attrs_int_values Array(Int64), - scope_attrs_float_keys Array(LowCardinality(String)), - scope_attrs_float_values Array(Float64), - scope_attrs_bool_keys Array(LowCardinality(String)), - scope_attrs_bool_values Array(Bool), - scope_attrs_bytes_keys Array(LowCardinality(String)), - scope_attrs_bytes_values Array(String), - - events_timestamps Array(DateTime64(9)), - events_names Array(String), - events_attrs_str_keys Array(Array(LowCardinality(String))), - events_attrs_str_values Array(Array(String)), - events_attrs_int_keys Array(Array(LowCardinality(String))), - events_attrs_int_values Array(Array(Int64)), - events_attrs_float_keys Array(Array(LowCardinality(String))), - events_attrs_float_values Array(Array(Float64)), - events_attrs_bool_keys Array(Array(LowCardinality(String))), - events_attrs_bool_values Array(Array(Bool)), - events_attrs_bytes_keys Array(Array(LowCardinality(String))), - events_attrs_bytes_values Array(Array(String)), - - links_trace_ids Array(UUID), - links_span_ids Array(UInt64), - links_tracestates Array(String), - links_attrs_str_keys Array(Array(LowCardinality(String))), - links_attrs_str_values Array(Array(String)), - links_attrs_int_keys Array(Array(LowCardinality(String))), - links_attrs_int_values Array(Array(Int64)), - links_attrs_float_keys Array(Array(LowCardinality(String))), - links_attrs_float_values Array(Array(Float64)), - links_attrs_bool_keys Array(Array(LowCardinality(String))), - links_attrs_bool_values Array(Array(Bool)), - links_attrs_bytes_keys Array(Array(LowCardinality(String))), - links_attrs_bytes_values Array(Array(String)), -) -ENGINE = MergeTree() -PRIMARY KEY (trace_id, span_id);` - kindDDL = `'KIND_UNSPECIFIED' = 0,'KIND_INTERNAL' = 1,'KIND_SERVER' = 2,'KIND_CLIENT' = 3,'KIND_PRODUCER' = 4,'KIND_CONSUMER' = 5` - - tagsSchema = `CREATE TABLE IF NOT EXISTS %s -( - name LowCardinality(String), - value String, - value_type Enum8( - 'EMPTY' = 0, - 'STR' = 1, - 'INT' = 2, - 'DOUBLE' = 3, - 'BOOL' = 4, - 'MAP' = 5, - 'SLICE' = 6, - 'BYTES' = 7 - ) -) -ENGINE = MergeTree() -PRIMARY KEY (name);` - valueTypeDDL = `'EMPTY' = 0,'STR' = 1,'INT' = 2,'DOUBLE' = 3,'BOOL' = 4,'MAP' = 5,'SLICE' = 6,'BYTES' = 7` -) - // Tables define table names. type Tables struct { Spans string Tags string + + Points string } var defaultTables = Tables{ Spans: "traces_spans", Tags: "traces_tags", + + Points: "metrics_points", } type chClient interface { diff --git a/internal/chstorage/schema_metrics.go b/internal/chstorage/schema_metrics.go new file mode 100644 index 00000000..c56da942 --- /dev/null +++ b/internal/chstorage/schema_metrics.go @@ -0,0 +1,61 @@ +package chstorage + +import ( + "errors" + "strconv" + + "github.com/go-faster/jx" +) + +const ( + metricSchema = `CREATE TABLE IF NOT EXISTS %s +( + name LowCardinality(String), + ts DateTime64(9), + value Float64, + attributes String, + resource String +) +ENGINE = MergeTree() +ORDER BY ts;` +) + +func parseLabels(s string, to map[string]string) error { + d := jx.DecodeStr(s) + return d.ObjBytes(func(d *jx.Decoder, key []byte) error { + switch d.Next() { + case jx.String: + val, err := d.Str() + if err != nil { + return err + } + to[string(key)] = val + return nil + case jx.Number: + val, err := d.Num() + if err != nil { + return err + } + to[string(key)] = val.String() + return nil + case jx.Null: + return d.Null() + case jx.Bool: + val, err := d.Bool() + if err != nil { + return err + } + to[string(key)] = strconv.FormatBool(val) + return nil + case jx.Array, jx.Object: + val, err := d.Raw() + if err != nil { + return err + } + to[string(key)] = val.String() + return nil + default: + return errors.New("invalid type") + } + }) +} diff --git a/internal/chstorage/schema_traces.go b/internal/chstorage/schema_traces.go new file mode 100644 index 00000000..832e366d --- /dev/null +++ b/internal/chstorage/schema_traces.go @@ -0,0 +1,92 @@ +package chstorage + +const ( + spansSchema = `CREATE TABLE IF NOT EXISTS %s +( + trace_id UUID, + span_id UInt64, + trace_state String, + parent_span_id UInt64, + name LowCardinality(String), + kind Enum8(` + kindDDL + `), + start DateTime64(9), + end DateTime64(9), + attrs_str_keys Array(LowCardinality(String)), + attrs_str_values Array(String), + attrs_int_keys Array(LowCardinality(String)), + attrs_int_values Array(Int64), + attrs_float_keys Array(LowCardinality(String)), + attrs_float_values Array(Float64), + attrs_bool_keys Array(LowCardinality(String)), + attrs_bool_values Array(Bool), + attrs_bytes_keys Array(LowCardinality(String)), + attrs_bytes_values Array(String), + status_code Int32, + status_message String, + + batch_id UUID, + resource_attrs_str_keys Array(LowCardinality(String)), + resource_attrs_str_values Array(String), + resource_attrs_int_keys Array(LowCardinality(String)), + resource_attrs_int_values Array(Int64), + resource_attrs_float_keys Array(LowCardinality(String)), + resource_attrs_float_values Array(Float64), + resource_attrs_bool_keys Array(LowCardinality(String)), + resource_attrs_bool_values Array(Bool), + resource_attrs_bytes_keys Array(LowCardinality(String)), + resource_attrs_bytes_values Array(String), + + scope_name String, + scope_version String, + scope_attrs_str_keys Array(LowCardinality(String)), + scope_attrs_str_values Array(String), + scope_attrs_int_keys Array(LowCardinality(String)), + scope_attrs_int_values Array(Int64), + scope_attrs_float_keys Array(LowCardinality(String)), + scope_attrs_float_values Array(Float64), + scope_attrs_bool_keys Array(LowCardinality(String)), + scope_attrs_bool_values Array(Bool), + scope_attrs_bytes_keys Array(LowCardinality(String)), + scope_attrs_bytes_values Array(String), + + events_timestamps Array(DateTime64(9)), + events_names Array(String), + events_attrs_str_keys Array(Array(LowCardinality(String))), + events_attrs_str_values Array(Array(String)), + events_attrs_int_keys Array(Array(LowCardinality(String))), + events_attrs_int_values Array(Array(Int64)), + events_attrs_float_keys Array(Array(LowCardinality(String))), + events_attrs_float_values Array(Array(Float64)), + events_attrs_bool_keys Array(Array(LowCardinality(String))), + events_attrs_bool_values Array(Array(Bool)), + events_attrs_bytes_keys Array(Array(LowCardinality(String))), + events_attrs_bytes_values Array(Array(String)), + + links_trace_ids Array(UUID), + links_span_ids Array(UInt64), + links_tracestates Array(String), + links_attrs_str_keys Array(Array(LowCardinality(String))), + links_attrs_str_values Array(Array(String)), + links_attrs_int_keys Array(Array(LowCardinality(String))), + links_attrs_int_values Array(Array(Int64)), + links_attrs_float_keys Array(Array(LowCardinality(String))), + links_attrs_float_values Array(Array(Float64)), + links_attrs_bool_keys Array(Array(LowCardinality(String))), + links_attrs_bool_values Array(Array(Bool)), + links_attrs_bytes_keys Array(Array(LowCardinality(String))), + links_attrs_bytes_values Array(Array(String)), +) +ENGINE = MergeTree() +PRIMARY KEY (trace_id, span_id);` + kindDDL = `'KIND_UNSPECIFIED' = 0,'KIND_INTERNAL' = 1,'KIND_SERVER' = 2,'KIND_CLIENT' = 3,'KIND_PRODUCER' = 4,'KIND_CONSUMER' = 5` + + tagsSchema = `CREATE TABLE IF NOT EXISTS %s +( + name LowCardinality(String), + value String, + value_type Enum8(` + valueTypeDDL + `) +) +ENGINE = MergeTree() +PRIMARY KEY (name);` + valueTypeDDL = `'EMPTY' = 0,'STR' = 1,'INT' = 2,'DOUBLE' = 3,'BOOL' = 4,'MAP' = 5,'SLICE' = 6,'BYTES' = 7` +) From 2f0383fafc9fc202f4a689177f8d8b7ae2636ca4 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 19 Nov 2023 17:11:39 +0300 Subject: [PATCH 002/112] feat(dev): privision prometheus and node-exporter --- .../{clickhouse => ch}/docker-compose.yml | 41 +- dev/local/grafana/dashboards.yml | 8 + .../grafana/dashboards/node-exporter.json | 23775 ++++++++++++++++ .../datasources.yaml} | 21 +- dev/local/{ => grafana}/grafana.ini | 0 dev/local/prometheus/prometheus.yml | 29 + dev/local/ytsaurus/docker-compose.yml | 6 +- 7 files changed, 23873 insertions(+), 7 deletions(-) rename dev/local/{clickhouse => ch}/docker-compose.yml (57%) create mode 100644 dev/local/grafana/dashboards.yml create mode 100644 dev/local/grafana/dashboards/node-exporter.json rename dev/local/{grafana-datasources.yaml => grafana/datasources.yaml} (52%) rename dev/local/{ => grafana}/grafana.ini (100%) create mode 100644 dev/local/prometheus/prometheus.yml diff --git a/dev/local/clickhouse/docker-compose.yml b/dev/local/ch/docker-compose.yml similarity index 57% rename from dev/local/clickhouse/docker-compose.yml rename to dev/local/ch/docker-compose.yml index 75853ace..7b3007a6 100644 --- a/dev/local/clickhouse/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -1,4 +1,8 @@ version: "3" + +volumes: + prom_data: + services: clickhouse: image: clickhouse/clickhouse-server:23.4 @@ -23,6 +27,7 @@ services: - OTEL_RESOURCE_ATTRIBUTES=service.name=go-faster.oteldb ports: # Querier ports + - "9090:9090" # prometheus API - "3200:3200" # tempo REST API # - "9095:9095" # tempo gRPC API - "3100:3100" # loki REST API @@ -43,6 +48,36 @@ services: depends_on: - oteldb + # to scrape node-exporter + # also can remote-write to otelcol + prometheus: + image: prom/prometheus + command: + - '--config.file=/etc/prometheus/prometheus.yml' + ports: + - "9091:9090" + restart: unless-stopped + volumes: + - ../prometheus:/etc/prometheus + - prom_data:/prometheus + + # to produce some metrics + node-exporter: + image: prom/node-exporter + container_name: node-exporter + restart: unless-stopped + volumes: + - /proc:/host/proc:ro + - /sys:/host/sys:ro + - /:/rootfs:ro + command: + - '--path.procfs=/host/proc' + - '--path.rootfs=/rootfs' + - '--path.sysfs=/host/sys' + - '--collector.processes' + - '--web.max-requests=40' + - '--collector.filesystem.mount-points-exclude=^/(sys|proc|dev|host|etc)($$|/)' + grafana: image: "grafana/grafana:10.0.0" environment: @@ -54,5 +89,7 @@ services: ports: - "3000:3000" volumes: - - ../grafana.ini:/etc/grafana/grafana.ini - - ../grafana-datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml + - ../grafana/grafana.ini:/etc/grafana/grafana.ini + - ../grafana/datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml + - ../grafana/dashboards.yml:/etc/grafana/provisioning/dashboards/default.yml + - ../grafana/dashboards:/etc/grafana/dashboards:ro diff --git a/dev/local/grafana/dashboards.yml b/dev/local/grafana/dashboards.yml new file mode 100644 index 00000000..22603e5d --- /dev/null +++ b/dev/local/grafana/dashboards.yml @@ -0,0 +1,8 @@ +apiVersion: 1 + +providers: + - name: Default + folder: Default + type: file + options: + path: /etc/grafana/dashboards diff --git a/dev/local/grafana/dashboards/node-exporter.json b/dev/local/grafana/dashboards/node-exporter.json new file mode 100644 index 00000000..7c55f198 --- /dev/null +++ b/dev/local/grafana/dashboards/node-exporter.json @@ -0,0 +1,23775 @@ +{ + "annotations": { + "list": [ + { + "$$hashKey": "object:1058", + "builtIn": 1, + "datasource": { + "type": "datasource", + "uid": "grafana" + }, + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "target": { + "limit": 100, + "matchAny": false, + "tags": [], + "type": "dashboard" + }, + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "gnetId": 1860, + "graphTooltip": 1, + "links": [ + { + "icon": "external link", + "tags": [], + "targetBlank": true, + "title": "GitHub", + "type": "link", + "url": "https://github.com/rfmoz/grafana-dashboards" + }, + { + "icon": "external link", + "tags": [], + "targetBlank": true, + "title": "Grafana", + "type": "link", + "url": "https://grafana.com/grafana/dashboards/1860" + } + ], + "liveNow": false, + "panels": [ + { + "collapsed": false, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 261, + "panels": [], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Quick CPU / Mem / Disk", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Busy state of all CPU cores together", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 1, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "max": 100, + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "rgba(50, 172, 45, 0.97)", + "value": null + }, + { + "color": "rgba(237, 129, 40, 0.89)", + "value": 85 + }, + { + "color": "rgba(245, 54, 54, 0.9)", + "value": 95 + } + ] + }, + "unit": "percent" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 0, + "y": 1 + }, + "id": 20, + "links": [], + "options": { + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "(sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode!=\"idle\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))) * 100", + "hide": false, + "instant": true, + "intervalFactor": 1, + "legendFormat": "", + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "CPU Busy", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Busy state of all CPU cores together (5 min average)", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 1, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "max": 100, + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "rgba(50, 172, 45, 0.97)", + "value": null + }, + { + "color": "rgba(237, 129, 40, 0.89)", + "value": 85 + }, + { + "color": "rgba(245, 54, 54, 0.9)", + "value": 95 + } + ] + }, + "unit": "percent" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 3, + "y": 1 + }, + "id": 155, + "links": [], + "options": { + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "avg_over_time(node_load5{instance=\"$node\",job=\"$job\"}[$__rate_interval]) * 100 / on(instance) group_left sum by (instance)(irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval]))", + "format": "time_series", + "hide": false, + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "Sys Load (5m avg)", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Busy state of all CPU cores together (15 min average)", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 1, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "max": 100, + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "rgba(50, 172, 45, 0.97)", + "value": null + }, + { + "color": "rgba(237, 129, 40, 0.89)", + "value": 85 + }, + { + "color": "rgba(245, 54, 54, 0.9)", + "value": 95 + } + ] + }, + "unit": "percent" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 6, + "y": 1 + }, + "id": 19, + "links": [], + "options": { + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "avg_over_time(node_load15{instance=\"$node\",job=\"$job\"}[$__rate_interval]) * 100 / on(instance) group_left sum by (instance)(irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval]))", + "hide": false, + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "Sys Load (15m avg)", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Non available RAM memory", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 1, + "mappings": [], + "max": 100, + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "rgba(50, 172, 45, 0.97)", + "value": null + }, + { + "color": "rgba(237, 129, 40, 0.89)", + "value": 80 + }, + { + "color": "rgba(245, 54, 54, 0.9)", + "value": 90 + } + ] + }, + "unit": "percent" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 9, + "y": 1 + }, + "hideTimeOverride": false, + "id": 16, + "links": [], + "options": { + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "((avg_over_time(node_memory_MemTotal_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval]) - avg_over_time(node_memory_MemFree_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval])) / (avg_over_time(node_memory_MemTotal_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval]) )) * 100", + "format": "time_series", + "hide": true, + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "100 - ((avg_over_time(node_memory_MemAvailable_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval]) * 100) / avg_over_time(node_memory_MemTotal_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval]))", + "format": "time_series", + "hide": false, + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "B", + "step": 240 + } + ], + "title": "RAM Used", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Used Swap", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 1, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "max": 100, + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "rgba(50, 172, 45, 0.97)", + "value": null + }, + { + "color": "rgba(237, 129, 40, 0.89)", + "value": 10 + }, + { + "color": "rgba(245, 54, 54, 0.9)", + "value": 25 + } + ] + }, + "unit": "percent" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 12, + "y": 1 + }, + "id": 21, + "links": [], + "options": { + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "((avg_over_time(node_memory_SwapTotal_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval]) - avg_over_time(node_memory_SwapFree_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval])) / (avg_over_time(node_memory_SwapTotal_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval]) )) * 100", + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "SWAP Used", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Used Root FS", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 1, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "max": 100, + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "rgba(50, 172, 45, 0.97)", + "value": null + }, + { + "color": "rgba(237, 129, 40, 0.89)", + "value": 80 + }, + { + "color": "rgba(245, 54, 54, 0.9)", + "value": 90 + } + ] + }, + "unit": "percent" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 15, + "y": 1 + }, + "id": 154, + "links": [], + "options": { + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "100 - ((avg_over_time(node_filesystem_avail_bytes{instance=\"$node\",job=\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"}[$__rate_interval]) * 100) / avg_over_time(node_filesystem_size_bytes{instance=\"$node\",job=\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"}[$__rate_interval]))", + "format": "time_series", + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "Root FS Used", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total number of CPU cores", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 2, + "w": 2, + "x": 18, + "y": 1 + }, + "id": 14, + "links": [], + "maxDataPoints": 100, + "options": { + "colorMode": "none", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "count(count(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}) by (cpu))", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "CPU Cores", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "System uptime", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 1, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 2, + "w": 4, + "x": 20, + "y": 1 + }, + "hideTimeOverride": true, + "id": 15, + "links": [], + "maxDataPoints": 100, + "options": { + "colorMode": "none", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "node_time_seconds{instance=\"$node\",job=\"$job\"} - node_boot_time_seconds{instance=\"$node\",job=\"$job\"}", + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "Uptime", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total RootFS", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 0, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "rgba(50, 172, 45, 0.97)", + "value": null + }, + { + "color": "rgba(237, 129, 40, 0.89)", + "value": 70 + }, + { + "color": "rgba(245, 54, 54, 0.9)", + "value": 90 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 2, + "w": 2, + "x": 18, + "y": 3 + }, + "id": 23, + "links": [], + "maxDataPoints": 100, + "options": { + "colorMode": "none", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "node_filesystem_size_bytes{instance=\"$node\",job=\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"}", + "format": "time_series", + "hide": false, + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "RootFS Total", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total RAM", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 0, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 2, + "w": 2, + "x": 20, + "y": 3 + }, + "id": 75, + "links": [], + "maxDataPoints": 100, + "options": { + "colorMode": "none", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "node_memory_MemTotal_bytes{instance=\"$node\",job=\"$job\"}", + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "RAM Total", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total SWAP", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "decimals": 0, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "text": "N/A" + } + }, + "type": "special" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 2, + "w": 2, + "x": 22, + "y": 3 + }, + "id": 18, + "links": [], + "maxDataPoints": 100, + "options": { + "colorMode": "none", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "node_memory_SwapTotal_bytes{instance=\"$node\",job=\"$job\"}", + "instant": true, + "intervalFactor": 1, + "range": false, + "refId": "A", + "step": 240 + } + ], + "title": "SWAP Total", + "type": "stat" + }, + { + "collapsed": false, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 263, + "panels": [], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Basic CPU / Mem / Net / Disk", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Basic CPU info", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "smooth", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "percent" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Busy Iowait" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Idle" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Busy Iowait" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Idle" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Busy System" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Busy User" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Busy Other" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 6 + }, + "id": 77, + "links": [], + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true, + "width": 250 + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"system\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Busy System", + "range": true, + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"user\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Busy User", + "range": true, + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"iowait\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Busy Iowait", + "range": true, + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=~\".*irq\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Busy IRQs", + "range": true, + "refId": "D", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode!='idle',mode!='user',mode!='system',mode!='iowait',mode!='irq',mode!='softirq'}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Busy Other", + "range": true, + "refId": "E", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"idle\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Idle", + "range": true, + "refId": "F", + "step": 240 + } + ], + "title": "CPU Basic", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Basic memory usage", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "SWAP Used" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap Used" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + }, + { + "id": "custom.stacking", + "value": { + "group": false, + "mode": "normal" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM Cache + Buffer" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Available" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#DEDAF7", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + }, + { + "id": "custom.stacking", + "value": { + "group": false, + "mode": "normal" + } + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 6 + }, + "id": 78, + "links": [], + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_MemTotal_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "RAM Total", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_MemTotal_bytes{instance=\"$node\",job=\"$job\"} - node_memory_MemFree_bytes{instance=\"$node\",job=\"$job\"} - (node_memory_Cached_bytes{instance=\"$node\",job=\"$job\"} + node_memory_Buffers_bytes{instance=\"$node\",job=\"$job\"} + node_memory_SReclaimable_bytes{instance=\"$node\",job=\"$job\"})", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "RAM Used", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Cached_bytes{instance=\"$node\",job=\"$job\"} + node_memory_Buffers_bytes{instance=\"$node\",job=\"$job\"} + node_memory_SReclaimable_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "RAM Cache + Buffer", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_MemFree_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "RAM Free", + "refId": "D", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "(node_memory_SwapTotal_bytes{instance=\"$node\",job=\"$job\"} - node_memory_SwapFree_bytes{instance=\"$node\",job=\"$job\"})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "SWAP Used", + "refId": "E", + "step": 240 + } + ], + "title": "Memory Basic", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Basic network info per interface", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bps" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Recv_bytes_eth2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Recv_bytes_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Recv_drop_eth2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Recv_drop_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Recv_errs_eth2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Recv_errs_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CCA300", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Trans_bytes_eth2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Trans_bytes_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Trans_drop_eth2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Trans_drop_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Trans_errs_eth2" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Trans_errs_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CCA300", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "recv_bytes_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "recv_drop_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "recv_drop_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#967302", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "recv_errs_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "recv_errs_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "trans_bytes_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "trans_bytes_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "trans_drop_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "trans_drop_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#967302", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "trans_errs_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "trans_errs_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 13 + }, + "id": 74, + "links": [], + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_bytes_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])*8", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "recv {{device}}", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_bytes_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])*8", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "trans {{device}} ", + "refId": "B", + "step": 240 + } + ], + "title": "Network Traffic Basic", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Disk space used of all filesystems mounted", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "max": 100, + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "percent" + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 13 + }, + "id": 152, + "links": [], + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "100 - ((node_filesystem_avail_bytes{instance=\"$node\",job=\"$job\",device!~'rootfs'} * 100) / node_filesystem_size_bytes{instance=\"$node\",job=\"$job\",device!~'rootfs'})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{mountpoint}}", + "refId": "A", + "step": 240 + } + ], + "title": "Disk Space Used Basic", + "type": "timeseries" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 20 + }, + "id": 265, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "percentage", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 70, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "percent" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Idle - Waiting for something to happen" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Iowait - Waiting for I/O to complete" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Irq - Servicing interrupts" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Nice - Niced processes executing in user mode" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Softirq - Servicing softirqs" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Steal - Time spent in other operating systems when running in a virtualized environment" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCE2DE", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "System - Processes executing in kernel mode" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "User - Normal processes executing in user mode" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#5195CE", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 12, + "w": 12, + "x": 0, + "y": 23 + }, + "id": 3, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 250 + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"system\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "System - Processes executing in kernel mode", + "range": true, + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"user\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "User - Normal processes executing in user mode", + "range": true, + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"nice\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Nice - Niced processes executing in user mode", + "range": true, + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"iowait\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Iowait - Waiting for I/O to complete", + "range": true, + "refId": "E", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"irq\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Irq - Servicing interrupts", + "range": true, + "refId": "F", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"softirq\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Softirq - Servicing softirqs", + "range": true, + "refId": "G", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"steal\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Steal - Time spent in other operating systems when running in a virtualized environment", + "range": true, + "refId": "H", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\", mode=\"idle\"}[$__rate_interval])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Idle - Waiting for something to happen", + "range": true, + "refId": "J", + "step": 240 + } + ], + "title": "CPU", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap - Swap memory usage" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused - Free memory unassigned" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*Hardware Corrupted - *./" + }, + "properties": [ + { + "id": "custom.stacking", + "value": { + "group": false, + "mode": "normal" + } + } + ] + } + ] + }, + "gridPos": { + "h": 12, + "w": 12, + "x": 12, + "y": 23 + }, + "id": 24, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_MemTotal_bytes{instance=\"$node\",job=\"$job\"} - node_memory_MemFree_bytes{instance=\"$node\",job=\"$job\"} - node_memory_Buffers_bytes{instance=\"$node\",job=\"$job\"} - node_memory_Cached_bytes{instance=\"$node\",job=\"$job\"} - node_memory_Slab_bytes{instance=\"$node\",job=\"$job\"} - node_memory_PageTables_bytes{instance=\"$node\",job=\"$job\"} - node_memory_SwapCached_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Apps - Memory used by user-space applications", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_PageTables_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "PageTables - Memory used to map between virtual and physical memory addresses", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_SwapCached_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "SwapCache - Memory that keeps track of pages that have been fetched from swap but not yet been modified", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Slab_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Slab - Memory used by the kernel to cache data structures for its own use (caches like inode, dentry, etc)", + "refId": "D", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Cached_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Cache - Parked file data (file content) cache", + "refId": "E", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Buffers_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Buffers - Block device (e.g. harddisk) cache", + "refId": "F", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_MemFree_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Unused - Free memory unassigned", + "refId": "G", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "(node_memory_SwapTotal_bytes{instance=\"$node\",job=\"$job\"} - node_memory_SwapFree_bytes{instance=\"$node\",job=\"$job\"})", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Swap - Swap space used", + "refId": "H", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_HardwareCorrupted_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working", + "refId": "I", + "step": 240 + } + ], + "title": "Memory Stack", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bits out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bps" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "receive_packets_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "receive_packets_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "transmit_packets_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "transmit_packets_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 12, + "w": 12, + "x": 0, + "y": 35 + }, + "id": 84, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_bytes_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])*8", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Receive", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_bytes_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])*8", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Transmit", + "refId": "B", + "step": 240 + } + ], + "title": "Network Traffic", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 12, + "w": 12, + "x": 12, + "y": 35 + }, + "id": 156, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filesystem_size_bytes{instance=\"$node\",job=\"$job\",device!~'rootfs'} - node_filesystem_avail_bytes{instance=\"$node\",job=\"$job\",device!~'rootfs'}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{mountpoint}}", + "refId": "A", + "step": 240 + } + ], + "title": "Disk Space Used", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "IO read (-) / write (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "iops" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Read.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 12, + "w": 12, + "x": 0, + "y": 47 + }, + "id": 229, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_reads_completed_total{instance=\"$node\",job=\"$job\",device=~\"$diskdevices\"}[$__rate_interval])", + "intervalFactor": 4, + "legendFormat": "{{device}} - Reads completed", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_writes_completed_total{instance=\"$node\",job=\"$job\",device=~\"$diskdevices\"}[$__rate_interval])", + "intervalFactor": 1, + "legendFormat": "{{device}} - Writes completed", + "refId": "B", + "step": 240 + } + ], + "title": "Disk IOps", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes read (-) / write (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "Bps" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "io time" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*read*./" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byType", + "options": "time" + }, + "properties": [ + { + "id": "custom.axisPlacement", + "value": "hidden" + } + ] + } + ] + }, + "gridPos": { + "h": 12, + "w": 12, + "x": 12, + "y": 47 + }, + "id": 42, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_read_bytes_total{instance=\"$node\",job=\"$job\",device=~\"$diskdevices\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{device}} - Successfully read bytes", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_written_bytes_total{instance=\"$node\",job=\"$job\",device=~\"$diskdevices\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{device}} - Successfully written bytes", + "refId": "B", + "step": 240 + } + ], + "title": "I/O Usage Read / Write", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "%util", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 40, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "io time" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byType", + "options": "time" + }, + "properties": [ + { + "id": "custom.axisPlacement", + "value": "hidden" + } + ] + } + ] + }, + "gridPos": { + "h": 12, + "w": 12, + "x": 0, + "y": 59 + }, + "id": 127, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_io_time_seconds_total{instance=\"$node\",job=\"$job\",device=~\"$diskdevices\"} [$__rate_interval])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{device}}", + "refId": "A", + "step": 240 + } + ], + "title": "I/O Utilization", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "percentage", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "bars", + "fillOpacity": 70, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 3, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "max": 1, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/^Guest - /" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#5195ce", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/^GuestNice - /" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#c15c17", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 12, + "w": 12, + "x": 12, + "y": 59 + }, + "id": 319, + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_guest_seconds_total{instance=\"$node\",job=\"$job\", mode=\"user\"}[1m])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[1m])))", + "hide": false, + "legendFormat": "Guest - Time spent running a virtual CPU for a guest operating system", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum by(instance) (irate(node_cpu_guest_seconds_total{instance=\"$node\",job=\"$job\", mode=\"nice\"}[1m])) / on(instance) group_left sum by (instance)((irate(node_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[1m])))", + "hide": false, + "legendFormat": "GuestNice - Time spent running a niced guest (virtual CPU for guest operating system)", + "range": true, + "refId": "B" + } + ], + "title": "CPU spent seconds in guests (VMs)", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "CPU / Memory / Net / Disk", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 21 + }, + "id": 266, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 54 + }, + "id": 136, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Inactive_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Inactive - Memory which has been less recently used. It is more eligible to be reclaimed for other purposes", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Active_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Active - Memory that has been used more recently and usually not reclaimed unless absolutely necessary", + "refId": "B", + "step": 240 + } + ], + "title": "Memory Active / Inactive", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*CommitLimit - *./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 54 + }, + "id": 135, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Committed_AS_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Committed_AS - Amount of memory presently allocated on the system", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_CommitLimit_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "CommitLimit - Amount of memory currently available to be allocated on the system", + "refId": "B", + "step": 240 + } + ], + "title": "Memory Committed", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 64 + }, + "id": 191, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Inactive_file_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Inactive_file - File-backed memory on inactive LRU list", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Inactive_anon_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Inactive_anon - Anonymous and swap cache on inactive LRU list, including tmpfs (shmem)", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Active_file_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Active_file - File-backed memory on active LRU list", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Active_anon_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Active_anon - Anonymous and swap cache on active least-recently-used (LRU) list, including tmpfs", + "refId": "D", + "step": 240 + } + ], + "title": "Memory Active / Inactive Detail", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 64 + }, + "id": 130, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Writeback_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Writeback - Memory which is actively being written back to disk", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_WritebackTmp_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "WritebackTmp - Memory used by FUSE for temporary writeback buffers", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Dirty_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Dirty - Memory which is waiting to get written back to the disk", + "refId": "C", + "step": 240 + } + ], + "title": "Memory Writeback and Dirty", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages" + }, + "properties": [ + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages" + }, + "properties": [ + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 74 + }, + "id": 138, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Mapped_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Mapped - Used memory in mapped pages files which have been mapped, such as libraries", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Shmem_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Shmem - Used shared memory (shared between several processes, thus including RAM disks)", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_ShmemHugePages_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_ShmemPmdMapped_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "ShmemPmdMapped - Amount of shared (shmem/tmpfs) memory backed by huge pages", + "refId": "D", + "step": 240 + } + ], + "title": "Memory Shared and Mapped", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 74 + }, + "id": 131, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_SUnreclaim_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "SUnreclaim - Part of Slab, that cannot be reclaimed on memory pressure", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_SReclaimable_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "SReclaimable - Part of Slab, that might be reclaimed, such as caches", + "refId": "B", + "step": 240 + } + ], + "title": "Memory Slab", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 84 + }, + "id": 70, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_VmallocChunk_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "VmallocChunk - Largest contiguous block of vmalloc area which is free", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_VmallocTotal_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "VmallocTotal - Total size of vmalloc memory area", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_VmallocUsed_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "VmallocUsed - Amount of vmalloc area which is used", + "refId": "C", + "step": 240 + } + ], + "title": "Memory Vmalloc", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 84 + }, + "id": 159, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Bounce_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Bounce - Memory used for block device bounce buffers", + "refId": "A", + "step": 240 + } + ], + "title": "Memory Bounce", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*Inactive *./" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 94 + }, + "id": 129, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_AnonHugePages_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "AnonHugePages - Memory in anonymous huge pages", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_AnonPages_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "AnonPages - Memory in user pages not backed by files", + "refId": "B", + "step": 240 + } + ], + "title": "Memory Anonymous", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 94 + }, + "id": 160, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_KernelStack_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "KernelStack - Kernel memory stack. This is not reclaimable", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Percpu_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "PerCPU - Per CPU memory allocated dynamically by loadable modules", + "refId": "B", + "step": 240 + } + ], + "title": "Memory Kernel / CPU", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "pages", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 104 + }, + "id": 140, + "links": [], + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_HugePages_Free{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "HugePages_Free - Huge pages in the pool that are not yet allocated", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_HugePages_Rsvd{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "HugePages_Rsvd - Huge pages for which a commitment to allocate from the pool has been made, but no allocation has yet been made", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_HugePages_Surp{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "HugePages_Surp - Huge pages in the pool above the value in /proc/sys/vm/nr_hugepages", + "refId": "C", + "step": 240 + } + ], + "title": "Memory HugePages Counter", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 104 + }, + "id": 71, + "links": [], + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_HugePages_Total{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "HugePages - Total size of the pool of huge pages", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Hugepagesize_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Hugepagesize - Huge Page size", + "refId": "B", + "step": 240 + } + ], + "title": "Memory HugePages Size", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 114 + }, + "id": 128, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_DirectMap1G_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "DirectMap1G - Amount of pages mapped as this size", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_DirectMap2M_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "DirectMap2M - Amount of pages mapped as this size", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_DirectMap4k_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "DirectMap4K - Amount of pages mapped as this size", + "refId": "C", + "step": 240 + } + ], + "title": "Memory DirectMap", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 114 + }, + "id": 137, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Unevictable_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Unevictable - Amount of unevictable memory that can't be swapped out for a variety of reasons", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_Mlocked_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "MLocked - Size of pages locked to memory using the mlock() system call", + "refId": "B", + "step": 240 + } + ], + "title": "Memory Unevictable and MLocked", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 124 + }, + "id": 132, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_memory_NFS_Unstable_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "NFS Unstable - Memory in NFS pages sent to the server, but not yet committed to the storage", + "refId": "A", + "step": 240 + } + ], + "title": "Memory NFS", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Memory Meminfo", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 22 + }, + "id": 267, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "pages out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*out/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 41 + }, + "id": 176, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_vmstat_pgpgin{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Pagesin - Page in operations", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_vmstat_pgpgout{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Pagesout - Page out operations", + "refId": "B", + "step": 240 + } + ], + "title": "Memory Pages In / Out", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "pages out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*out/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 41 + }, + "id": 22, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_vmstat_pswpin{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Pswpin - Pages swapped in", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_vmstat_pswpout{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Pswpout - Pages swapped out", + "refId": "B", + "step": 240 + } + ], + "title": "Memory Pages Swap In / Out", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "faults", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Apps" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#629E51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A437C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#CFFAFF", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "RAM_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#806EB7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#2F575E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Unused" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Pgfault - Page major and minor fault operations" + }, + "properties": [ + { + "id": "custom.fillOpacity", + "value": 0 + }, + { + "id": "custom.stacking", + "value": { + "group": false, + "mode": "normal" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 51 + }, + "id": 175, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 350 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_vmstat_pgfault{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Pgfault - Page major and minor fault operations", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_vmstat_pgmajfault{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Pgmajfault - Major page fault operations", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_vmstat_pgfault{instance=\"$node\",job=\"$job\"}[$__rate_interval]) - irate(node_vmstat_pgmajfault{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Pgminfault - Minor page fault operations", + "refId": "C", + "step": 240 + } + ], + "title": "Memory Page Faults", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#99440A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Buffers" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#58140C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6D1F62", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cached" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Committed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#508642", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Dirty" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#B7DBAB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mapped" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "PageTables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Page_Tables" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slab_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Swap_Cache" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C15C17", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#511749", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total RAM + Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#052B51", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total Swap" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "VmallocUsed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 51 + }, + "id": 307, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_vmstat_oom_kill{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "oom killer invocations ", + "refId": "A", + "step": 240 + } + ], + "title": "OOM Killer", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Memory Vmstat", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 23 + }, + "id": 293, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "seconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "s" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Variation*./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 56 + }, + "id": 260, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_timex_estimated_error_seconds{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Estimated error in seconds", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_timex_offset_seconds{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Time offset in between local system and reference clock", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_timex_maxerror_seconds{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Maximum error in seconds", + "refId": "C", + "step": 240 + } + ], + "title": "Time Synchronized Drift", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 56 + }, + "id": 291, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_timex_loop_time_constant{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Phase-locked loop time adjust", + "refId": "A", + "step": 240 + } + ], + "title": "Time PLL Adjust", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Variation*./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 66 + }, + "id": 168, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_timex_sync_status{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Is clock synchronized to a reliable server (1 = yes, 0 = no)", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_timex_frequency_adjustment_ratio{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Local clock frequency adjustment", + "refId": "B", + "step": 240 + } + ], + "title": "Time Synchronized Status", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "seconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 66 + }, + "id": 294, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_timex_tick_seconds{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Seconds between clock ticks", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_timex_tai_offset_seconds{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "International Atomic Time (TAI) offset", + "refId": "B", + "step": 240 + } + ], + "title": "Time Misc", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "System Timesync", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 24 + }, + "id": 312, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 43 + }, + "id": 62, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_procs_blocked{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Processes blocked waiting for I/O to complete", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_procs_running{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Processes in runnable state", + "refId": "B", + "step": 240 + } + ], + "title": "Processes Status", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 43 + }, + "id": 315, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_processes_state{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ state }}", + "refId": "A", + "step": 240 + } + ], + "title": "Processes State", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "forks / sec", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 53 + }, + "id": 148, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_forks_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Processes forks second", + "refId": "A", + "step": 240 + } + ], + "title": "Processes Forks", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "decbytes" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Max.*/" + }, + "properties": [ + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 53 + }, + "id": 149, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(process_virtual_memory_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Processes virtual memory size in bytes", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "process_resident_memory_max_bytes{instance=\"$node\",job=\"$job\"}", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Maximum amount of virtual memory available in bytes", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(process_virtual_memory_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Processes virtual memory size in bytes", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(process_virtual_memory_max_bytes{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Maximum amount of virtual memory available in bytes", + "refId": "D", + "step": 240 + } + ], + "title": "Processes Memory", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "PIDs limit" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F2495C", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 63 + }, + "id": 313, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_processes_pids{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Number of PIDs", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_processes_max_processes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "PIDs limit", + "refId": "B", + "step": 240 + } + ], + "title": "PIDs Number and Limit", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "seconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "s" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*waiting.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 63 + }, + "id": 305, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_schedstat_running_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "CPU {{ cpu }} - seconds spent running a process", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_schedstat_waiting_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "CPU {{ cpu }} - seconds spent by processing waiting for this CPU", + "refId": "B", + "step": 240 + } + ], + "title": "Process schedule stats Running / Waiting", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Threads limit" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F2495C", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 73 + }, + "id": 314, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_processes_threads{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Allocated threads", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_processes_max_threads{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Threads limit", + "refId": "B", + "step": 240 + } + ], + "title": "Threads Number and Limit", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "System Processes", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 25 + }, + "id": 269, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 26 + }, + "id": 8, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_context_switches_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Context switches", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_intr_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Interrupts", + "refId": "B", + "step": 240 + } + ], + "title": "Context Switches / Interrupts", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 26 + }, + "id": 7, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_load1{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 4, + "legendFormat": "Load 1m", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_load5{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 4, + "legendFormat": "Load 5m", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_load15{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 4, + "legendFormat": "Load 15m", + "refId": "C", + "step": 240 + } + ], + "title": "System Load", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "hertz" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Max" + }, + "properties": [ + { + "id": "custom.lineStyle", + "value": { + "dash": [ + 10, + 10 + ], + "fill": "dash" + } + }, + { + "id": "color", + "value": { + "fixedColor": "blue", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 10 + }, + { + "id": "custom.hideFrom", + "value": { + "legend": true, + "tooltip": false, + "viz": false + } + }, + { + "id": "custom.fillBelowTo", + "value": "Min" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Min" + }, + "properties": [ + { + "id": "custom.lineStyle", + "value": { + "dash": [ + 10, + 10 + ], + "fill": "dash" + } + }, + { + "id": "color", + "value": { + "fixedColor": "blue", + "mode": "fixed" + } + }, + { + "id": "custom.hideFrom", + "value": { + "legend": true, + "tooltip": false, + "viz": false + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 36 + }, + "id": 321, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "node_cpu_scaling_frequency_hertz{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "CPU {{ cpu }}", + "range": true, + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg(node_cpu_scaling_frequency_max_hertz{instance=\"$node\",job=\"$job\"})", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Max", + "range": true, + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg(node_cpu_scaling_frequency_min_hertz{instance=\"$node\",job=\"$job\"})", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Min", + "range": true, + "refId": "C", + "step": 240 + } + ], + "title": "CPU Frequency Scaling", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "https://docs.kernel.org/accounting/psi.html", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Memory some" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-red", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Memory full" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "light-red", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "I/O some" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-blue", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "I/O full" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "light-blue", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 36 + }, + "id": 322, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(node_pressure_cpu_waiting_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "CPU some", + "range": true, + "refId": "CPU some", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(node_pressure_memory_waiting_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Memory some", + "range": true, + "refId": "Memory some", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(node_pressure_memory_stalled_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Memory full", + "range": true, + "refId": "Memory full", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(node_pressure_io_waiting_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "I/O some", + "range": true, + "refId": "I/O some", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(node_pressure_io_stalled_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "I/O full", + "range": true, + "refId": "I/O full", + "step": 240 + } + ], + "title": "Pressure Stall Information", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Critical*./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*Max*./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 46 + }, + "id": 259, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_interrupts_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ type }} - {{ info }}", + "refId": "A", + "step": 240 + } + ], + "title": "Interrupts Detail", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 46 + }, + "id": 306, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_schedstat_timeslices_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "CPU {{ cpu }}", + "refId": "A", + "step": 240 + } + ], + "title": "Schedule timeslices executed by each cpu", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 56 + }, + "id": 151, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_entropy_available_bits{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Entropy available to random number generators", + "refId": "A", + "step": 240 + } + ], + "title": "Entropy", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "seconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 56 + }, + "id": 308, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(process_cpu_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Time spent", + "refId": "A", + "step": 240 + } + ], + "title": "CPU time spent in user and system contexts", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Max*./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 66 + }, + "id": 64, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "process_max_fds{instance=\"$node\",job=\"$job\"}", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Maximum open file descriptors", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "process_open_fds{instance=\"$node\",job=\"$job\"}", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Open file descriptors", + "refId": "B", + "step": 240 + } + ], + "title": "File Descriptors", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "System Misc", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 26 + }, + "id": 304, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "temperature", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "celsius" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Critical*./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*Max*./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 59 + }, + "id": 158, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_hwmon_temp_celsius{instance=\"$node\",job=\"$job\"} * on(chip) group_left(chip_name) node_hwmon_chip_names", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ chip_name }} {{ sensor }} temp", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_hwmon_temp_crit_alarm_celsius{instance=\"$node\",job=\"$job\"} * on(chip) group_left(chip_name) node_hwmon_chip_names", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ chip_name }} {{ sensor }} Critical Alarm", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_hwmon_temp_crit_celsius{instance=\"$node\",job=\"$job\"} * on(chip) group_left(chip_name) node_hwmon_chip_names", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ chip_name }} {{ sensor }} Critical", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_hwmon_temp_crit_hyst_celsius{instance=\"$node\",job=\"$job\"} * on(chip) group_left(chip_name) node_hwmon_chip_names", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ chip_name }} {{ sensor }} Critical Historical", + "refId": "D", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_hwmon_temp_max_celsius{instance=\"$node\",job=\"$job\"} * on(chip) group_left(chip_name) node_hwmon_chip_names", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ chip_name }} {{ sensor }} Max", + "refId": "E", + "step": 240 + } + ], + "title": "Hardware temperature monitor", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Max*./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 59 + }, + "id": 300, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_cooling_device_cur_state{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "Current {{ name }} in {{ type }}", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_cooling_device_max_state{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Max {{ name }} in {{ type }}", + "refId": "B", + "step": 240 + } + ], + "title": "Throttle cooling device", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 69 + }, + "id": 302, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_power_supply_online{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ power_supply }} online", + "refId": "A", + "step": 240 + } + ], + "title": "Power supply", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Hardware Misc", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 27 + }, + "id": 296, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 46 + }, + "id": 297, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_systemd_socket_accepted_connections_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ name }} Connections", + "refId": "A", + "step": 240 + } + ], + "title": "Systemd Sockets", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Failed" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F2495C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Inactive" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FF9830", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#73BF69", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Deactivating" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FFCB7D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Activating" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#C8F2C2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 46 + }, + "id": 298, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_systemd_units{instance=\"$node\",job=\"$job\",state=\"activating\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Activating", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_systemd_units{instance=\"$node\",job=\"$job\",state=\"active\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Active", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_systemd_units{instance=\"$node\",job=\"$job\",state=\"deactivating\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Deactivating", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_systemd_units{instance=\"$node\",job=\"$job\",state=\"failed\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Failed", + "refId": "D", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_systemd_units{instance=\"$node\",job=\"$job\",state=\"inactive\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Inactive", + "refId": "E", + "step": 240 + } + ], + "title": "Systemd Units State", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Systemd", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 28 + }, + "id": 270, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "The number (after merges) of I/O requests completed per second for the device", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "IO read (-) / write (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "iops" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Read.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 47 + }, + "id": 9, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_reads_completed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "intervalFactor": 4, + "legendFormat": "{{device}} - Reads completed", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_writes_completed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "intervalFactor": 1, + "legendFormat": "{{device}} - Writes completed", + "refId": "B", + "step": 240 + } + ], + "title": "Disk IOps Completed", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "The number of bytes read from or written to the device per second", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes read (-) / write (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "Bps" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Read.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 47 + }, + "id": 33, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_read_bytes_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 4, + "legendFormat": "{{device}} - Read bytes", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_written_bytes_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Written bytes", + "refId": "B", + "step": 240 + } + ], + "title": "Disk R/W Data", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "The average time for requests issued to the device to be served. This includes the time spent by the requests in queue and the time spent servicing them.", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time. read (-) / write (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "s" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Read.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 57 + }, + "id": 37, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_read_time_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval]) / irate(node_disk_reads_completed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "hide": false, + "interval": "", + "intervalFactor": 4, + "legendFormat": "{{device}} - Read wait time avg", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_write_time_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval]) / irate(node_disk_writes_completed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{device}} - Write wait time avg", + "refId": "B", + "step": 240 + } + ], + "title": "Disk Average Wait Time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "The average queue length of the requests that were issued to the device", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "aqu-sz", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "none" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 57 + }, + "id": 35, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_io_time_weighted_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "intervalFactor": 4, + "legendFormat": "{{device}}", + "refId": "A", + "step": 240 + } + ], + "title": "Average Queue Size", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "The number of read and write requests merged per second that were queued to the device", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "I/Os", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "iops" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Read.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 67 + }, + "id": 133, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_reads_merged_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "intervalFactor": 1, + "legendFormat": "{{device}} - Read merged", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_writes_merged_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "intervalFactor": 1, + "legendFormat": "{{device}} - Write merged", + "refId": "B", + "step": 240 + } + ], + "title": "Disk R/W Merged", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Percentage of elapsed time during which I/O requests were issued to the device (bandwidth utilization for the device). Device saturation occurs when this value is close to 100% for devices serving requests serially. But for devices serving requests in parallel, such as RAID arrays and modern SSDs, this number does not reflect their performance limits.", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "%util", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 67 + }, + "id": 36, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_io_time_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "intervalFactor": 4, + "legendFormat": "{{device}} - IO", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_discard_time_seconds_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "intervalFactor": 4, + "legendFormat": "{{device}} - discard", + "refId": "B", + "step": 240 + } + ], + "title": "Time Spent Doing I/Os", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "The number of outstanding requests at the instant the sample was taken. Incremented as requests are given to appropriate struct request_queue and decremented as they finish.", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Outstanding req.", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "none" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 77 + }, + "id": 34, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_disk_io_now{instance=\"$node\",job=\"$job\"}", + "interval": "", + "intervalFactor": 4, + "legendFormat": "{{device}} - IO now", + "refId": "A", + "step": 240 + } + ], + "title": "Instantaneous Queue Size", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "IOs", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "iops" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EAB839", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#6ED0E0", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EF843C", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#584477", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda2_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BA43A9", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sda3_.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F4D598", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#0A50A1", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#BF1B00", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdb3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0752D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#962D82", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#614D93", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdc3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#9AC48A", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#65C5DB", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9934E", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#EA6460", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde1.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E0F9D7", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sdd2.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#FCEACA", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*sde3.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F9E2D2", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 77 + }, + "id": 301, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_discards_completed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "intervalFactor": 4, + "legendFormat": "{{device}} - Discards completed", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_disk_discards_merged_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{device}} - Discards merged", + "refId": "B", + "step": 240 + } + ], + "title": "Disk IOps Discards completed / merged", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Storage Disk", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 29 + }, + "id": 271, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 62 + }, + "id": 43, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filesystem_avail_bytes{instance=\"$node\",job=\"$job\",device!~'rootfs'}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{mountpoint}} - Available", + "metric": "", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filesystem_free_bytes{instance=\"$node\",job=\"$job\",device!~'rootfs'}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{mountpoint}} - Free", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filesystem_size_bytes{instance=\"$node\",job=\"$job\",device!~'rootfs'}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{mountpoint}} - Size", + "refId": "C", + "step": 240 + } + ], + "title": "Filesystem space available", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "file nodes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 62 + }, + "id": 41, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filesystem_files_free{instance=\"$node\",job=\"$job\",device!~'rootfs'}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{mountpoint}} - Free file nodes", + "refId": "A", + "step": 240 + } + ], + "title": "File Nodes Free", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "files", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 72 + }, + "id": 28, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filefd_maximum{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 4, + "legendFormat": "Max open files", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filefd_allocated{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Open files", + "refId": "B", + "step": 240 + } + ], + "title": "File Descriptor", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "file Nodes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 72 + }, + "id": 219, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filesystem_files{instance=\"$node\",job=\"$job\",device!~'rootfs'}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{mountpoint}} - File nodes total", + "refId": "A", + "step": 240 + } + ], + "title": "File Nodes Size", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "max": 1, + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "/ ReadOnly" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 82 + }, + "id": 44, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filesystem_readonly{instance=\"$node\",job=\"$job\",device!~'rootfs'}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{mountpoint}} - ReadOnly", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_filesystem_device_error{instance=\"$node\",job=\"$job\",device!~'rootfs',fstype!~'tmpfs'}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{mountpoint}} - Device error", + "refId": "B", + "step": 240 + } + ], + "title": "Filesystem in ReadOnly / Error", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Storage Filesystem", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 30 + }, + "id": 272, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packets out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "pps" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "receive_packets_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "receive_packets_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "transmit_packets_eth0" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#7EB26D", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "transmit_packets_lo" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#E24D42", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 47 + }, + "id": 60, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_packets_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{device}} - Receive", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_packets_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{device}} - Transmit", + "refId": "B", + "step": 240 + } + ], + "title": "Network Traffic by Packets", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packets out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "pps" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 47 + }, + "id": 142, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_errs_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Receive errors", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_errs_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Rransmit errors", + "refId": "B", + "step": 240 + } + ], + "title": "Network Traffic Errors", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packets out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "pps" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 57 + }, + "id": 143, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_drop_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Receive drop", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_drop_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Transmit drop", + "refId": "B", + "step": 240 + } + ], + "title": "Network Traffic Drop", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packets out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "pps" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 57 + }, + "id": 141, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_compressed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Receive compressed", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_compressed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Transmit compressed", + "refId": "B", + "step": 240 + } + ], + "title": "Network Traffic Compressed", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packets out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "pps" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 67 + }, + "id": 146, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_multicast_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Receive multicast", + "refId": "A", + "step": 240 + } + ], + "title": "Network Traffic Multicast", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packets out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "pps" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 67 + }, + "id": 144, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_fifo_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Receive fifo", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_fifo_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Transmit fifo", + "refId": "B", + "step": 240 + } + ], + "title": "Network Traffic Fifo", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packets out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "pps" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 77 + }, + "id": 145, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_receive_frame_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{device}} - Receive frame", + "refId": "A", + "step": 240 + } + ], + "title": "Network Traffic Frame", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 77 + }, + "id": 231, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_carrier_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Statistic transmit_carrier", + "refId": "A", + "step": 240 + } + ], + "title": "Network Traffic Carrier", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Trans.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 87 + }, + "id": 232, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_network_transmit_colls_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{device}} - Transmit colls", + "refId": "A", + "step": 240 + } + ], + "title": "Network Traffic Colls", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "entries", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "NF conntrack limit" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 87 + }, + "id": 61, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_nf_conntrack_entries{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "NF conntrack entries", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_nf_conntrack_entries_limit{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "NF conntrack limit", + "refId": "B", + "step": 240 + } + ], + "title": "NF Contrack", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Entries", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 97 + }, + "id": 230, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_arp_entries{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{ device }} - ARP entries", + "refId": "A", + "step": 240 + } + ], + "title": "ARP Entries", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "decimals": 0, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 97 + }, + "id": 288, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_network_mtu_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{ device }} - Bytes", + "refId": "A", + "step": 240 + } + ], + "title": "MTU", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "decimals": 0, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 107 + }, + "id": 280, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_network_speed_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{ device }} - Speed", + "refId": "A", + "step": 240 + } + ], + "title": "Speed", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packets", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "decimals": 0, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 107 + }, + "id": 289, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_network_transmit_queue_length{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{ device }} - Interface transmit queue length", + "refId": "A", + "step": 240 + } + ], + "title": "Queue Length", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "packetes drop (-) / process (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Dropped.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 117 + }, + "id": 290, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_softnet_processed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "CPU {{cpu}} - Processed", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_softnet_dropped_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "CPU {{cpu}} - Dropped", + "refId": "B", + "step": 240 + } + ], + "title": "Softnet Packets", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 117 + }, + "id": 310, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_softnet_times_squeezed_total{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "CPU {{cpu}} - Squeezed", + "refId": "A", + "step": 240 + } + ], + "title": "Softnet Out of Quota", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 127 + }, + "id": 309, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_network_up{operstate=\"up\",instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{interface}} - Operational state UP", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_network_carrier{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "instant": false, + "legendFormat": "{{device}} - Physical link state", + "refId": "B" + } + ], + "title": "Network Operational Status", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Network Traffic", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 31 + }, + "id": 273, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 48 + }, + "id": 63, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_TCP_alloc{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "TCP_alloc - Allocated sockets", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_TCP_inuse{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "TCP_inuse - Tcp sockets currently in use", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_TCP_mem{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "TCP_mem - Used memory for tcp", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_TCP_orphan{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "TCP_orphan - Orphan sockets", + "refId": "D", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_TCP_tw{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "TCP_tw - Sockets waiting close", + "refId": "E", + "step": 240 + } + ], + "title": "Sockstat TCP", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 48 + }, + "id": 124, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_UDPLITE_inuse{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "UDPLITE_inuse - Udplite sockets currently in use", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_UDP_inuse{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "UDP_inuse - Udp sockets currently in use", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_UDP_mem{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "UDP_mem - Used memory for udp", + "refId": "C", + "step": 240 + } + ], + "title": "Sockstat UDP", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 58 + }, + "id": 125, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_FRAG_inuse{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "FRAG_inuse - Frag sockets currently in use", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_RAW_inuse{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "RAW_inuse - Raw sockets currently in use", + "refId": "C", + "step": 240 + } + ], + "title": "Sockstat FRAG / RAW", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "bytes", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 58 + }, + "id": 220, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_TCP_mem_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "mem_bytes - TCP sockets in that state", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_UDP_mem_bytes{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "mem_bytes - UDP sockets in that state", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_FRAG_memory{instance=\"$node\",job=\"$job\"}", + "interval": "", + "intervalFactor": 1, + "legendFormat": "FRAG_memory - Used memory for frag", + "refId": "C" + } + ], + "title": "Sockstat Memory Size", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "sockets", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 68 + }, + "id": 126, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_sockstat_sockets_used{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Sockets_used - Sockets currently in use", + "refId": "A", + "step": 240 + } + ], + "title": "Sockstat Used", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Network Sockstat", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 32 + }, + "id": 274, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "octets out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Out.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 49 + }, + "id": 221, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_IpExt_InOctets{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "InOctets - Received octets", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_IpExt_OutOctets{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "OutOctets - Sent octets", + "refId": "B", + "step": 240 + } + ], + "title": "Netstat IP In / Out Octets", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "datagrams", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 49 + }, + "id": 81, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true, + "width": 300 + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Ip_Forwarding{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Forwarding - IP forwarding", + "refId": "A", + "step": 240 + } + ], + "title": "Netstat IP Forwarding", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "messages out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Out.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 59 + }, + "id": 115, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Icmp_InMsgs{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "InMsgs - Messages which the entity received. Note that this counter includes all those counted by icmpInErrors", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Icmp_OutMsgs{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "OutMsgs - Messages which this entity attempted to send. Note that this counter includes all those counted by icmpOutErrors", + "refId": "B", + "step": 240 + } + ], + "title": "ICMP In / Out", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "messages out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Out.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 59 + }, + "id": 50, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Icmp_InErrors{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "InErrors - Messages which the entity received but determined as having ICMP-specific errors (bad ICMP checksums, bad length, etc.)", + "refId": "A", + "step": 240 + } + ], + "title": "ICMP Errors", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "datagrams out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Out.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*Snd.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 69 + }, + "id": 55, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Udp_InDatagrams{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "InDatagrams - Datagrams received", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Udp_OutDatagrams{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "OutDatagrams - Datagrams sent", + "refId": "B", + "step": 240 + } + ], + "title": "UDP In / Out", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "datagrams", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 69 + }, + "id": 109, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Udp_InErrors{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "InErrors - UDP Datagrams that could not be delivered to an application", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Udp_NoPorts{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "NoPorts - UDP Datagrams received on a port with no listener", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_UdpLite_InErrors{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "legendFormat": "InErrors Lite - UDPLite Datagrams that could not be delivered to an application", + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Udp_RcvbufErrors{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "RcvbufErrors - UDP buffer errors received", + "refId": "D", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Udp_SndbufErrors{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "SndbufErrors - UDP buffer errors send", + "refId": "E", + "step": 240 + } + ], + "title": "UDP Errors", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "datagrams out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Out.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": "/.*Snd.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 79 + }, + "id": 299, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Tcp_InSegs{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "InSegs - Segments received, including those received in error. This count includes segments received on currently established connections", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Tcp_OutSegs{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "OutSegs - Segments sent, including those on current connections but excluding those containing only retransmitted octets", + "refId": "B", + "step": 240 + } + ], + "title": "TCP In / Out", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 79 + }, + "id": 104, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_TcpExt_ListenOverflows{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "ListenOverflows - Times the listen queue of a socket overflowed", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_TcpExt_ListenDrops{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "ListenDrops - SYNs to LISTEN sockets ignored", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_TcpExt_TCPSynRetrans{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "TCPSynRetrans - SYN-SYN/ACK retransmits to break down retransmissions in SYN, fast/timeout retransmits", + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Tcp_RetransSegs{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "legendFormat": "RetransSegs - Segments retransmitted - that is, the number of TCP segments transmitted containing one or more previously transmitted octets", + "refId": "D" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Tcp_InErrs{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "legendFormat": "InErrs - Segments received in error (e.g., bad TCP checksums)", + "refId": "E" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Tcp_OutRsts{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "interval": "", + "legendFormat": "OutRsts - Segments sent with RST flag", + "refId": "F" + } + ], + "title": "TCP Errors", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "connections", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*MaxConn *./" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#890F02", + "mode": "fixed" + } + }, + { + "id": "custom.fillOpacity", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 89 + }, + "id": 85, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_netstat_Tcp_CurrEstab{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "CurrEstab - TCP connections for which the current state is either ESTABLISHED or CLOSE- WAIT", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_netstat_Tcp_MaxConn{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "MaxConn - Limit on the total number of TCP connections the entity can support (Dynamic is \"-1\")", + "refId": "B", + "step": 240 + } + ], + "title": "TCP Connections", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter out (-) / in (+)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*Sent.*/" + }, + "properties": [ + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 89 + }, + "id": 91, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_TcpExt_SyncookiesFailed{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "SyncookiesFailed - Invalid SYN cookies received", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_TcpExt_SyncookiesRecv{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "SyncookiesRecv - SYN cookies received", + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_TcpExt_SyncookiesSent{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "SyncookiesSent - SYN cookies sent", + "refId": "C", + "step": 240 + } + ], + "title": "TCP SynCookie", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "connections", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 99 + }, + "id": 82, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Tcp_ActiveOpens{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "ActiveOpens - TCP connections that have made a direct transition to the SYN-SENT state from the CLOSED state", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "irate(node_netstat_Tcp_PassiveOpens{instance=\"$node\",job=\"$job\"}[$__rate_interval])", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "PassiveOpens - TCP connections that have made a direct transition to the SYN-RCVD state from the LISTEN state", + "refId": "B", + "step": 240 + } + ], + "title": "TCP Direct Transition", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Enable with --collector.tcpstat argument on node-exporter", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "connections", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 99 + }, + "id": 320, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "node_tcp_connection_states{state=\"established\",instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "established - TCP sockets in established state", + "range": true, + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "node_tcp_connection_states{state=\"fin_wait2\",instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "fin_wait2 - TCP sockets in fin_wait2 state", + "range": true, + "refId": "B", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "node_tcp_connection_states{state=\"listen\",instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "listen - TCP sockets in listen state", + "range": true, + "refId": "C", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "node_tcp_connection_states{state=\"time_wait\",instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "time_wait - TCP sockets in time_wait state", + "range": true, + "refId": "D", + "step": 240 + } + ], + "title": "TCP Stat", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Network Netstat", + "type": "row" + }, + { + "collapsed": true, + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 33 + }, + "id": 279, + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "seconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 0, + "y": 66 + }, + "id": 40, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_scrape_collector_duration_seconds{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{collector}} - Scrape duration", + "refId": "A", + "step": 240 + } + ], + "title": "Node Exporter Scrape Time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "counter", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineStyle": { + "fill": "solid" + }, + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "links": [], + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": "/.*error.*/" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "#F2495C", + "mode": "fixed" + } + }, + { + "id": "custom.transform", + "value": "negative-Y" + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 12, + "x": 12, + "y": 66 + }, + "id": 157, + "links": [], + "options": { + "legend": { + "calcs": [ + "mean", + "lastNotNull", + "max", + "min" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "9.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_scrape_collector_success{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{collector}} - Scrape success", + "refId": "A", + "step": 240 + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "expr": "node_textfile_scrape_error{instance=\"$node\",job=\"$job\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{collector}} - Scrape textfile error (1 = true)", + "refId": "B", + "step": 240 + } + ], + "title": "Node Exporter Scrape", + "type": "timeseries" + } + ], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "000000001" + }, + "refId": "A" + } + ], + "title": "Node Exporter", + "type": "row" + } + ], + "refresh": "5s", + "revision": 1, + "schemaVersion": 38, + "style": "dark", + "tags": [ + "linux" + ], + "templating": { + "list": [ + { + "current": { + "selected": false, + "text": "Prometheus", + "value": "Prometheus" + }, + "hide": 0, + "includeAll": false, + "label": "datasource", + "multi": false, + "name": "DS_PROMETHEUS", + "options": [], + "query": "prometheus", + "queryValue": "", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "type": "datasource" + }, + { + "current": { + "selected": false, + "text": "node-exporter", + "value": "node-exporter" + }, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "definition": "", + "hide": 0, + "includeAll": false, + "label": "Job", + "multi": false, + "name": "job", + "options": [], + "query": { + "query": "label_values(node_uname_info, job)", + "refId": "Prometheus-job-Variable-Query" + }, + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "sort": 1, + "tagValuesQuery": "", + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "current": { + "selected": false, + "text": "node-exporter:9100", + "value": "node-exporter:9100" + }, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "definition": "label_values(node_uname_info{job=\"$job\"}, instance)", + "hide": 0, + "includeAll": false, + "label": "Host", + "multi": false, + "name": "node", + "options": [], + "query": { + "query": "label_values(node_uname_info{job=\"$job\"}, instance)", + "refId": "Prometheus-node-Variable-Query" + }, + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "sort": 1, + "tagValuesQuery": "", + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "current": { + "selected": false, + "text": "[a-z]+|nvme[0-9]+n[0-9]+|mmcblk[0-9]+", + "value": "[a-z]+|nvme[0-9]+n[0-9]+|mmcblk[0-9]+" + }, + "hide": 2, + "includeAll": false, + "multi": false, + "name": "diskdevices", + "options": [ + { + "selected": true, + "text": "[a-z]+|nvme[0-9]+n[0-9]+|mmcblk[0-9]+", + "value": "[a-z]+|nvme[0-9]+n[0-9]+|mmcblk[0-9]+" + } + ], + "query": "[a-z]+|nvme[0-9]+n[0-9]+|mmcblk[0-9]+", + "skipUrlSync": false, + "type": "custom" + } + ] + }, + "time": { + "from": "now-5m", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m" + ], + "time_options": [ + "5m", + "15m", + "1h" + ] + }, + "timezone": "browser", + "title": "Node Exporter Full", + "uid": "node-exporter", + "version": 2, + "weekStart": "" +} \ No newline at end of file diff --git a/dev/local/grafana-datasources.yaml b/dev/local/grafana/datasources.yaml similarity index 52% rename from dev/local/grafana-datasources.yaml rename to dev/local/grafana/datasources.yaml index 42aca7d0..76cc3492 100644 --- a/dev/local/grafana-datasources.yaml +++ b/dev/local/grafana/datasources.yaml @@ -6,12 +6,12 @@ datasources: access: proxy orgId: 1 url: http://oteldb:3200 - isDefault: true uid: tempo jsonData: httpMethod: GET serviceMap: - datasourceUid: prometheus + datasourceUid: promoteldb + - name: Loki type: loki access: proxy @@ -21,4 +21,19 @@ datasources: jsonData: httpMethod: GET serviceMap: - datasourceUid: prometheus + datasourceUid: promoteldb + + - name: Prometheus (oteldb) + type: prometheus + access: proxy + orgId: 1 + url: http://oteldb:9090 + uid: promoteldb + + - name: Prometheus + type: prometheus + access: proxy + orgId: 1 + url: http://prometheus:9090 + uid: prometheus + isDefault: true diff --git a/dev/local/grafana.ini b/dev/local/grafana/grafana.ini similarity index 100% rename from dev/local/grafana.ini rename to dev/local/grafana/grafana.ini diff --git a/dev/local/prometheus/prometheus.yml b/dev/local/prometheus/prometheus.yml new file mode 100644 index 00000000..d7899212 --- /dev/null +++ b/dev/local/prometheus/prometheus.yml @@ -0,0 +1,29 @@ +global: + scrape_interval: 5s + scrape_timeout: 2s + evaluation_interval: 5s + +alerting: + alertmanagers: + - static_configs: + - targets: [] + scheme: http + timeout: 10s + api_version: v1 + +scrape_configs: + - job_name: prometheus + honor_timestamps: true + metrics_path: /metrics + scheme: http + static_configs: + - targets: + - localhost:9090 + + - job_name: node-exporter + honor_timestamps: true + metrics_path: /metrics + scheme: http + static_configs: + - targets: + - node-exporter:9100 \ No newline at end of file diff --git a/dev/local/ytsaurus/docker-compose.yml b/dev/local/ytsaurus/docker-compose.yml index d19ea4a9..58a204e0 100644 --- a/dev/local/ytsaurus/docker-compose.yml +++ b/dev/local/ytsaurus/docker-compose.yml @@ -76,5 +76,7 @@ services: ports: - "3000:3000" volumes: - - ../grafana.ini:/etc/grafana/grafana.ini - - ../grafana-datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml + - ../grafana/grafana.ini:/etc/grafana/grafana.ini + - ../grafana/datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml + - ../grafana/dashboards.yml:/etc/grafana/provisioning/dashboards/default.yml + - ../grafana/dashboards:/etc/grafana/dashboards:ro From e212949c94980734ee2ef6bcee51bca58a7216a2 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 20 Nov 2023 14:06:33 +0300 Subject: [PATCH 003/112] feat(dev): setup otelcol --- dev/local/ch/clickhouse.xml | 37 + dev/local/ch/docker-compose.yml | 26 + .../dashboards/clickhouse-cluster.json | 1778 +++++++++++++ .../grafana/dashboards/clickhouse-data.json | 2279 +++++++++++++++++ dev/local/grafana/datasources.yaml | 7 + dev/local/otelcol.yml | 60 + dev/local/prometheus/prometheus.yml | 25 +- 7 files changed, 4190 insertions(+), 22 deletions(-) create mode 100644 dev/local/ch/clickhouse.xml create mode 100644 dev/local/grafana/dashboards/clickhouse-cluster.json create mode 100644 dev/local/grafana/dashboards/clickhouse-data.json create mode 100644 dev/local/otelcol.yml diff --git a/dev/local/ch/clickhouse.xml b/dev/local/ch/clickhouse.xml new file mode 100644 index 00000000..813166d2 --- /dev/null +++ b/dev/local/ch/clickhouse.xml @@ -0,0 +1,37 @@ + + 0.0.0.0 + + information + true + + + + + /metrics + 8080 + true + true + true + + + + + + engine MergeTree + order by (start_time_us, trace_id) + ttl toDateTime(finish_time_us/1000000) + toIntervalMinute(15) + + system + opentelemetry_span_log
+ 1000 +
+
\ No newline at end of file diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index 7b3007a6..98ed3031 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -10,6 +10,8 @@ services: - "9009:9009" - "8123:8123" - "9000:9000" + volumes: + - ./clickhouse.xml:/etc/clickhouse-server/config.d/monitoring.xml oteldb: image: go-faster/oteldb @@ -54,6 +56,7 @@ services: image: prom/prometheus command: - '--config.file=/etc/prometheus/prometheus.yml' + - '--web.enable-remote-write-receiver' ports: - "9091:9090" restart: unless-stopped @@ -61,6 +64,24 @@ services: - ../prometheus:/etc/prometheus - prom_data:/prometheus + # https://opentelemetry.io/docs/collector/installation/#docker-compose + otelcol: + image: ghcr.io/open-telemetry/opentelemetry-collector-releases/opentelemetry-collector-contrib:0.89.0 + # ports: + # - 1888:1888 # pprof extension + # - 8888:8888 # Prometheus metrics exposed by the Collector + # - 8889:8889 # Prometheus exporter metrics + # - 13133:13133 # health_check extension + # - 14317:4317 # OTLP gRPC receiver + # - 14318:4318 # OTLP http receiver + # - 55679:55679 # zpages extension + volumes: + - ../otelcol.yml:/etc/otelcol-contrib/config.yaml + depends_on: + - prometheus + - oteldb + - node-exporter + # to produce some metrics node-exporter: image: prom/node-exporter @@ -86,6 +107,7 @@ services: - GF_AUTH_DISABLE_LOGIN_FORM=true - GF_FEATURE_TOGGLES_ENABLE=traceqlEditor - GF_LOG_LEVEL=debug + - GF_INSTALL_PLUGINS=grafana-clickhouse-datasource ports: - "3000:3000" volumes: @@ -93,3 +115,7 @@ services: - ../grafana/datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml - ../grafana/dashboards.yml:/etc/grafana/provisioning/dashboards/default.yml - ../grafana/dashboards:/etc/grafana/dashboards:ro + depends_on: + - oteldb + - prometheus + - otelcol diff --git a/dev/local/grafana/dashboards/clickhouse-cluster.json b/dev/local/grafana/dashboards/clickhouse-cluster.json new file mode 100644 index 00000000..1856a74e --- /dev/null +++ b/dev/local/grafana/dashboards/clickhouse-cluster.json @@ -0,0 +1,1778 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": { + "type": "datasource", + "uid": "grafana" + }, + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "target": { + "limit": 100, + "matchAny": false, + "tags": [], + "type": "dashboard" + }, + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "graphTooltip": 0, + "id": 7, + "links": [], + "liveNow": false, + "panels": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-blue", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Version" + }, + "properties": [ + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 0, + "y": 0 + }, + "id": 2, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "/.*/", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT version()", + "refId": "A" + } + ], + "transformations": [ + { + "id": "convertFieldType", + "options": { + "conversions": [ + { + "destinationType": "string", + "targetField": "Version" + } + ], + "fields": {} + } + }, + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "version()": "Version" + } + } + } + ], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-blue", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "uptime" + }, + "properties": [ + { + "id": "unit", + "value": "s" + } + ] + } + ] + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 4, + "y": 0 + }, + "id": 4, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT uptime() as uptime", + "refId": "A" + } + ], + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "uptime": "Server uptime" + } + } + } + ], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-blue", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 8, + "y": 0 + }, + "id": 6, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT count() as \"Number of databases\" FROM system.databases WHERE name IN (${database:singlequote})", + "refId": "A" + } + ], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-blue", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 12, + "y": 0 + }, + "id": 7, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT count() as \"Number of tables\" FROM system.tables WHERE database IN (${database:singlequote})", + "refId": "A" + } + ], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-blue", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 16, + "y": 0 + }, + "id": 8, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT sum(total_rows) as \"Number of rows\" FROM system.tables WHERE database IN (${database:singlequote});", + "refId": "A" + } + ], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-blue", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 4, + "x": 20, + "y": 0 + }, + "id": 9, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT count() as \"Number of columns\" FROM system.columns WHERE database IN (${database:singlequote});", + "refId": "A" + } + ], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "center", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Is local" + }, + "properties": [ + { + "id": "mappings", + "value": [ + { + "options": { + "0": { + "color": "light-red", + "index": 0, + "text": "remote" + }, + "1": { + "color": "light-green", + "index": 1, + "text": "local" + } + }, + "type": "value" + } + ] + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Errors count" + }, + "properties": [ + { + "id": "color", + "value": { + "mode": "continuous-GrYlRd" + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "gradient", + "type": "color-background" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Slowdowns count" + }, + "properties": [ + { + "id": "color", + "value": { + "mode": "continuous-GrYlRd" + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "gradient", + "type": "color-background" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Cluster" + }, + "properties": [ + { + "id": "custom.filterable", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Host name" + }, + "properties": [ + { + "id": "custom.filterable", + "value": true + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 20, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT cluster, shard_num, replica_num, host_name, host_address, port, is_local, errors_count, slowdowns_count FROM system.clusters;\n", + "refId": "A" + } + ], + "title": "Cluster Overview", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "cluster": "Cluster", + "errors_count": "Errors count", + "host_address": "Host address", + "host_name": "Host name", + "is_local": "Is local", + "port": "Port", + "replica_num": "Replicated number", + "shard_num": "Shard number", + "slowdowns_count": "Slowdowns count" + } + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "percentage", + "steps": [ + { + "color": "green", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "progress" + }, + "properties": [ + { + "id": "unit", + "value": "percent" + }, + { + "id": "color", + "value": { + "mode": "continuous-RdYlGr" + } + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 5, + "x": 0, + "y": 15 + }, + "id": 13, + "options": { + "displayMode": "lcd", + "minVizHeight": 10, + "minVizWidth": 0, + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": true + }, + "showUnfilled": true, + "valueMode": "color" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(database, '.', table) as db_table, round(100 * progress, 1) \"progress\" FROM system.merges WHERE database IN (${database:singlequote}) ORDER BY progress DESC LIMIT 5;\n", + "refId": "A" + } + ], + "title": "Merge progress per table", + "type": "bargauge" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "center", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Database" + }, + "properties": [ + { + "id": "custom.width", + "value": 78 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Table" + }, + "properties": [ + { + "id": "custom.width", + "value": 200 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Elapsed" + }, + "properties": [ + { + "id": "custom.width", + "value": 75 + }, + { + "id": "unit", + "value": "s" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Progress" + }, + "properties": [ + { + "id": "custom.width", + "value": 82 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mutation" + }, + "properties": [ + { + "id": "custom.width", + "value": 100 + }, + { + "id": "unit", + "value": "bool" + }, + { + "id": "color", + "value": { + "mode": "thresholds" + } + }, + { + "id": "thresholds", + "value": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Partition id" + }, + "properties": [ + { + "id": "custom.width", + "value": 92 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Target path" + }, + "properties": [ + { + "id": "custom.width", + "value": 396 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Num parts" + }, + "properties": [ + { + "id": "custom.width", + "value": 90 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "total_size_compressed" + }, + "properties": [ + { + "id": "custom.width", + "value": 185 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Columns written" + }, + "properties": [ + { + "id": "custom.width", + "value": 198 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total compressed size" + }, + "properties": [ + { + "id": "custom.width", + "value": 166 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Compressed size" + }, + "properties": [ + { + "id": "custom.width", + "value": 138 + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 19, + "x": 5, + "y": 15 + }, + "id": 11, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(database, '.', table) as db_table, round(elapsed, 1) \"elapsed\", round(100 * progress, 1) \"progress\", is_mutation, partition_id, result_part_path, source_part_paths, num_parts, formatReadableSize(total_size_bytes_compressed) \"total_size_compressed\", formatReadableSize(bytes_read_uncompressed) \"read_uncompressed\", formatReadableSize(bytes_written_uncompressed) \"written_uncompressed\", columns_written, formatReadableSize(memory_usage) \"memory_usage\", thread_id FROM system.merges WHERE database IN (${database:singlequote});", + "refId": "Merges" + } + ], + "title": "Current merges", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": { + "partition_id": true, + "read_uncompressed": true, + "source_part_paths": true, + "thread_id": true, + "written_uncompressed": true + }, + "indexByName": {}, + "renameByName": { + "columns_written": "Columns written", + "database": "Database", + "db_table": "Table", + "elapsed": "Elapsed", + "is_mutation": "Mutation", + "memory_usage": "Memory usage", + "num_parts": "Num parts", + "partition_id": "Partition id", + "progress": "Progress", + "result_part_path": "Target path", + "table": "Table", + "total_size_compressed": "Compressed size" + } + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "percentage", + "steps": [ + { + "color": "green", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "parts_remaining" + }, + "properties": [ + { + "id": "min", + "value": 0 + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 5, + "x": 0, + "y": 22 + }, + "id": 14, + "options": { + "displayMode": "lcd", + "minVizHeight": 10, + "minVizWidth": 0, + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": true + }, + "showUnfilled": true, + "valueMode": "color" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(database, '.', table, ' - ', mutation_id) as db_table, length(parts_to_do_names) as parts_remaining FROM system.mutations WHERE parts_remaining > 0 AND database IN (${database:singlequote}) ORDER BY parts_remaining DESC;", + "refId": "A" + } + ], + "title": "Mutations parts remaining", + "type": "bargauge" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "center", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Is completed" + }, + "properties": [ + { + "id": "custom.width", + "value": 100 + }, + { + "id": "unit", + "value": "bool" + }, + { + "id": "color", + "value": { + "mode": "thresholds" + } + }, + { + "id": "thresholds", + "value": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "semi-dark-orange", + "value": 0 + }, + { + "color": "semi-dark-green", + "value": 1 + } + ] + } + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Database" + }, + "properties": [ + { + "id": "custom.width", + "value": 86 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Table" + }, + "properties": [ + { + "id": "custom.width", + "value": 194 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Fail time" + }, + "properties": [ + { + "id": "custom.width", + "value": 201 + }, + { + "id": "custom.align", + "value": "center" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Result" + }, + "properties": [ + { + "id": "custom.width", + "value": 133 + }, + { + "id": "mappings", + "value": [ + { + "options": { + "failure": { + "color": "light-red", + "index": 1, + "text": "failure" + }, + "success": { + "color": "semi-dark-green", + "index": 0, + "text": "success" + } + }, + "type": "value" + } + ] + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "gradient", + "type": "color-background" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Reason" + }, + "properties": [ + { + "id": "custom.width", + "value": 163 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Command" + }, + "properties": [ + { + "id": "custom.width", + "value": 300 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Mutation id" + }, + "properties": [ + { + "id": "custom.width", + "value": 157 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Create time" + }, + "properties": [ + { + "id": "custom.width", + "value": 194 + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 19, + "x": 5, + "y": 22 + }, + "id": 15, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(database, '.', table) as db_table, mutation_id, command, create_time, parts_to_do_names, is_done, latest_failed_part, if(latest_fail_time = '1970-01-01 01:00:00', 'success', 'failure') as success, if(latest_fail_time = '1970-01-01 01:00:00', '-', CAST(latest_fail_time, 'String')) as fail_time, latest_fail_reason FROM system.mutations WHERE database IN (${database:singlequote}) ORDER BY is_done ASC, create_time DESC LIMIT 10", + "refId": "Merges" + } + ], + "title": "Current mutations", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": { + "latest_failed_part": true, + "partition_id": true, + "parts_to_do_names": true, + "read_uncompressed": true, + "source_part_paths": true, + "thread_id": true, + "written_uncompressed": true + }, + "indexByName": { + "command": 5, + "create_time": 2, + "db_table": 0, + "fail_time": 9, + "is_done": 3, + "latest_fail_reason": 6, + "latest_failed_part": 8, + "mutation_id": 1, + "parts_to_do_names": 7, + "success": 4 + }, + "renameByName": { + "columns_written": "Columns written", + "command": "Command", + "create_time": "Create time", + "database": "Database", + "db_table": "Table", + "elapsed": "Elapsed", + "fail_time": "Fail time", + "is_done": "Is completed", + "is_mutation": "Mutation", + "latest_fail_reason": "Reason", + "latest_fail_time": "Fail time", + "memory_usage": "Memory usage", + "mutation_id": "Mutation id", + "num_parts": "Num parts", + "partition_id": "Partition id", + "parts_to_do_names": "Pending parts", + "progress": "Progress", + "result_part_path": "Target path", + "success": "Result", + "table": "Table", + "total_size_compressed": "Compressed size" + } + } + }, + { + "id": "convertFieldType", + "options": { + "conversions": [], + "fields": {} + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 5, + "x": 0, + "y": 29 + }, + "id": 18, + "options": { + "displayMode": "lcd", + "minVizHeight": 10, + "minVizWidth": 0, + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showUnfilled": true, + "valueMode": "color" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(database, '.', table) as db_table, queue_size FROM system.replicas WHERE database IN (${database:singlequote}) ORDER BY absolute_delay DESC LIMIT 10", + "refId": "A" + } + ], + "title": "Replicated tables by delay", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "absolute_delay": "Delay", + "db_table": "Table", + "inserts_in_queue": "Inserts in queue", + "is_leader": "Leader", + "is_readonly": "Readonly", + "merges_in_queue": "Merges in queue", + "queue_size": "Queue size" + } + } + } + ], + "type": "bargauge" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Table" + }, + "properties": [ + { + "id": "custom.width", + "value": 200 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Leader" + }, + "properties": [ + { + "id": "custom.width", + "value": 122 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Readonly" + }, + "properties": [ + { + "id": "custom.width", + "value": 138 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Delay" + }, + "properties": [ + { + "id": "custom.width", + "value": 108 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Queue size" + }, + "properties": [ + { + "id": "custom.width", + "value": 113 + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 19, + "x": 5, + "y": 29 + }, + "id": 17, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(database, '.', table) as db_table, is_leader, is_readonly, absolute_delay, queue_size, inserts_in_queue, merges_in_queue FROM system.replicas WHERE database IN (${database:singlequote}) ORDER BY absolute_delay DESC LIMIT 10", + "refId": "A" + } + ], + "title": "Replicated tables by delay", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "absolute_delay": "Delay", + "db_table": "Table", + "inserts_in_queue": "Inserts in queue", + "is_leader": "Leader", + "is_readonly": "Readonly", + "merges_in_queue": "Merges in queue", + "queue_size": "Queue size" + } + } + } + ], + "type": "table" + } + ], + "refresh": "", + "schemaVersion": 38, + "style": "dark", + "tags": [], + "templating": { + "list": [ + { + "current": { + "selected": false, + "text": "ClickHouse", + "value": "ClickHouse" + }, + "hide": 0, + "includeAll": false, + "label": "ClickHouse instance", + "multi": false, + "name": "datasource", + "options": [], + "query": "grafana-clickhouse-datasource", + "queryValue": "", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "type": "datasource" + }, + { + "current": { + "selected": false, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "definition": "SELECT name FROM system.databases;", + "hide": 0, + "includeAll": true, + "label": "Database", + "multi": false, + "name": "database", + "options": [], + "query": "SELECT name FROM system.databases;", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "type": "query" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "y-Ka8y37k" + }, + "filters": [], + "hide": 0, + "name": "filters", + "skipUrlSync": false, + "type": "adhoc" + } + ] + }, + "time": { + "from": "now-6h", + "to": "now" + }, + "timepicker": {}, + "timezone": "", + "title": "ClickHouse - Cluster Analysis", + "uid": "bf9293d7-811a-41ef-acbc-03407bb47429", + "version": 1, + "weekStart": "" +} \ No newline at end of file diff --git a/dev/local/grafana/dashboards/clickhouse-data.json b/dev/local/grafana/dashboards/clickhouse-data.json new file mode 100644 index 00000000..4887f1b0 --- /dev/null +++ b/dev/local/grafana/dashboards/clickhouse-data.json @@ -0,0 +1,2279 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": { + "type": "datasource", + "uid": "grafana" + }, + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "target": { + "limit": 100, + "matchAny": false, + "tags": [], + "type": "dashboard" + }, + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "graphTooltip": 0, + "id": 4, + "links": [], + "liveNow": false, + "panels": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-blue", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Version" + }, + "properties": [ + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, + "gridPos": { + "h": 6, + "w": 3, + "x": 0, + "y": 0 + }, + "id": 4, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "/.*/", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT version()", + "refId": "A" + } + ], + "transformations": [ + { + "id": "convertFieldType", + "options": { + "conversions": [ + { + "destinationType": "string", + "targetField": "Version" + } + ], + "fields": {} + } + }, + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "version()": "Version" + } + } + } + ], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-blue", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "uptime" + }, + "properties": [ + { + "id": "unit", + "value": "s" + } + ] + } + ] + }, + "gridPos": { + "h": 6, + "w": 3, + "x": 3, + "y": 0 + }, + "id": 2, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT uptime() as uptime", + "refId": "A" + } + ], + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "uptime": "Server uptime" + } + } + } + ], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "light-blue", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 3, + "x": 6, + "y": 0 + }, + "id": 22, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "value_and_name" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT sum(total_rows) as \"Total rows\" FROM system.tables WHERE database IN (${database}) AND name IN (${table})", + "refId": "A" + } + ], + "transformations": [], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "light-blue", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 3, + "x": 9, + "y": 0 + }, + "id": 23, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "value_and_name" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT count() as \"Total columns\" FROM system.columns WHERE database IN (${database}) AND table IN (${table})", + "refId": "A" + } + ], + "transformations": [], + "type": "stat" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "percentage", + "steps": [ + { + "color": "green", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Used" + }, + "properties": [ + { + "id": "unit", + "value": "percentunit" + }, + { + "id": "custom.cellOptions", + "value": { + "mode": "lcd", + "type": "gauge" + } + }, + { + "id": "max", + "value": 1 + }, + { + "id": "min", + "value": 0 + }, + { + "id": "thresholds", + "value": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "semi-dark-green", + "value": 0.6 + }, + { + "color": "#EAB839", + "value": 0.7 + }, + { + "color": "semi-dark-orange", + "value": 0.75 + }, + { + "color": "semi-dark-red", + "value": 0.8 + } + ] + } + }, + { + "id": "custom.width", + "value": 357 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Name" + }, + "properties": [ + { + "id": "custom.width", + "value": 74 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Free space" + }, + "properties": [ + { + "id": "custom.width", + "value": 96 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Path" + }, + "properties": [ + { + "id": "custom.width", + "value": 277 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total space" + }, + "properties": [ + { + "id": "custom.width", + "value": 103 + } + ] + } + ] + }, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 9, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT\n name as Name,\n path as Path,\n formatReadableSize(free_space) as Free,\n formatReadableSize(total_space) as Total,\n 1 - free_space/total_space as Used\nFROM system.disks", + "refId": "A" + } + ], + "title": "Disk usage", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "Free": "Free space", + "Name": "", + "Path": "", + "Total": "Total space", + "Used": "" + } + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-GrYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 0, + "y": 6 + }, + "id": 25, + "options": { + "displayMode": "lcd", + "minVizHeight": 10, + "minVizWidth": 0, + "orientation": "horizontal", + "reduceOptions": { + "calcs": [], + "fields": "", + "values": true + }, + "showUnfilled": true, + "valueMode": "color" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(table.database, '.', name) as name,\n table_stats.total_rows as total_rows\nFROM system.tables table\n LEFT JOIN ( SELECT table,\n database,\n sum(rows) as total_rows\nFROM system.parts\nWHERE table IN (${table}) AND active AND database IN (${database}) \nGROUP BY table, database\n ) AS table_stats ON table.name = table_stats.table AND table.database = table_stats.database\nORDER BY total_rows DESC\nLIMIT 10", + "refId": "A" + } + ], + "title": "Top tables by rows", + "type": "bargauge" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-GrYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 7, + "x": 8, + "y": 6 + }, + "id": 26, + "options": { + "displayMode": "lcd", + "minVizHeight": 10, + "minVizWidth": 0, + "orientation": "horizontal", + "reduceOptions": { + "calcs": [], + "fields": "", + "values": true + }, + "showUnfilled": true, + "valueMode": "color" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(table.database, '.', name) as name,\n col_stats.col_count as total_columns\nFROM system.tables table\n LEFT JOIN (SELECT database, table, count() as col_count FROM system.columns GROUP BY table, database) as col_stats\n ON table.name = col_stats.table AND col_stats.database = table.database\nWHERE database IN (${database}) AND name != '' AND table IN (${table}) AND name != '' ORDER BY total_columns DESC LIMIT 10;\n", + "refId": "A" + } + ], + "title": "Top tables by columns", + "type": "bargauge" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisGridShow": false, + "axisLabel": "", + "axisPlacement": "auto", + "axisSoftMin": 0, + "fillOpacity": 61, + "gradientMode": "hue", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineWidth": 1, + "scaleDistribution": { + "type": "linear" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 4, + "x": 15, + "y": 6 + }, + "id": 12, + "options": { + "barRadius": 0, + "barWidth": 1, + "fullHighlight": false, + "groupWidth": 0.7, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "orientation": "auto", + "showValue": "auto", + "stacking": "none", + "text": {}, + "tooltip": { + "mode": "single", + "sort": "none" + }, + "xTickLabelRotation": 0, + "xTickLabelSpacing": 0 + }, + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT engine, count() \"Number of databases\" FROM system.databases WHERE name IN (${database}) GROUP BY engine ", + "refId": "A" + } + ], + "title": "Database engines", + "type": "barchart" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisGridShow": false, + "axisLabel": "", + "axisPlacement": "auto", + "axisSoftMin": 0, + "fillOpacity": 61, + "gradientMode": "hue", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineWidth": 1, + "scaleDistribution": { + "type": "linear" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 5, + "x": 19, + "y": 6 + }, + "id": 11, + "options": { + "barRadius": 0, + "barWidth": 0.97, + "fullHighlight": false, + "groupWidth": 0.7, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "orientation": "auto", + "showValue": "auto", + "stacking": "none", + "tooltip": { + "mode": "single", + "sort": "none" + }, + "xTickLabelRotation": 0, + "xTickLabelSpacing": 0 + }, + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT engine, count() \"Number of tables\" FROM system.tables WHERE database IN (${database}) AND notLike(engine,'System%') AND name IN (${table}) GROUP BY engine ORDER BY count() DESC", + "refId": "A" + } + ], + "title": "Table engines", + "type": "barchart" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "percentage", + "steps": [ + { + "color": "green", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Engine" + }, + "properties": [ + { + "id": "custom.width", + "value": 200 + }, + { + "id": "custom.filterable", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Number of tables" + }, + "properties": [ + { + "id": "custom.width", + "value": 128 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total rows" + }, + "properties": [ + { + "id": "custom.width", + "value": 113 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Column count" + }, + "properties": [ + { + "id": "custom.width", + "value": 126 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Part count" + }, + "properties": [ + { + "id": "custom.width", + "value": 98 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Database" + }, + "properties": [ + { + "id": "custom.filterable", + "value": true + }, + { + "id": "custom.width", + "value": 205 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Partition count" + }, + "properties": [ + { + "id": "custom.width", + "value": 143 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Size on disk" + }, + "properties": [ + { + "id": "custom.width", + "value": 203 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 15, + "x": 0, + "y": 15 + }, + "id": 6, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT name,\n engine,\n tables,\n partitions,\n parts,\n formatReadableSize(bytes_on_disk) \"disk_size\",\n col_count,\n total_rows,\n formatReadableSize(data_uncompressed_bytes) as \"uncompressed_size\"\nFROM system.databases db\n LEFT JOIN ( SELECT database,\n uniq(table) \"tables\",\n uniq(table, partition) \"partitions\",\n count() AS parts,\n sum(bytes_on_disk) \"bytes_on_disk\",\n sum(data_compressed_bytes) as \"data_compressed_bytes\",\n sum(rows) as total_rows,\n max(col_count) as \"col_count\"\n FROM system.parts AS parts\n JOIN (SELECT database, count() as col_count\n FROM system.columns\n WHERE database IN (${database}) AND table IN (${table})\n GROUP BY database) as col_stats\n ON parts.database = col_stats.database\n WHERE database IN (${database}) AND active AND table IN (${table})\n GROUP BY database) AS db_stats ON db.name = db_stats.database\nWHERE database IN (${database}) AND lower(name) != 'information_schema'\nORDER BY bytes_on_disk DESC\nLIMIT 10;", + "refId": "A" + } + ], + "title": "Database summary", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": { + "col_count": 4, + "disk_size": 7, + "engine": 1, + "name": 0, + "partitions": 5, + "parts": 6, + "tables": 2, + "total_rows": 3, + "uncompressed_size": 8 + }, + "renameByName": { + "col_count": "Column count", + "disk_size": "Size on disk", + "engine": "Engine", + "name": "Database", + "partitions": "Partition count", + "parts": "Part count", + "tables": "Number of tables", + "total_rows": "Total rows", + "uncompressed_size": "Uncompressed size" + } + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Source" + }, + "properties": [ + { + "id": "custom.width", + "value": 85 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Type" + }, + "properties": [ + { + "id": "custom.width", + "value": 64 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Status" + }, + "properties": [ + { + "id": "custom.width", + "value": 71 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 9, + "x": 15, + "y": 15 + }, + "id": 14, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT source, type, status, count() \"count\" FROM system.dictionaries GROUP BY source, type, status ORDER BY status DESC, source", + "refId": "A" + } + ], + "title": "Dictionaries", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "count": "Usages", + "source": "Source", + "status": "Status", + "type": "Type" + } + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "filterable": false, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "percentage", + "steps": [ + { + "color": "green", + "value": null + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Engine" + }, + "properties": [ + { + "id": "custom.width", + "value": 86 + }, + { + "id": "custom.filterable", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Total rows" + }, + "properties": [ + { + "id": "custom.width", + "value": 116 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Column count" + }, + "properties": [ + { + "id": "custom.width", + "value": 156 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Partition count" + }, + "properties": [ + { + "id": "custom.width", + "value": 138 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Part count" + }, + "properties": [ + { + "id": "custom.width", + "value": 113 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Size on disk" + }, + "properties": [ + { + "id": "custom.width", + "value": 145 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Row Count" + }, + "properties": [ + { + "id": "custom.width", + "value": 109 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Database" + }, + "properties": [ + { + "id": "custom.width", + "value": 286 + }, + { + "id": "custom.filterable", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Table" + }, + "properties": [ + { + "id": "custom.width", + "value": 236 + }, + { + "id": "custom.filterable", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Uncompressed size" + }, + "properties": [ + { + "id": "custom.width", + "value": 171 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 15, + "x": 0, + "y": 25 + }, + "id": 7, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [ + { + "desc": true, + "displayName": "Column count" + } + ] + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT name,\n table.database,\n engine,\n partitions,\n parts,\n formatReadableSize(bytes_on_disk) \"disk_size\",\n col_count,\n table_stats.total_rows,\n formatReadableSize(data_uncompressed_bytes) as \"uncompressed_size\"\nFROM system.tables table\n LEFT JOIN ( SELECT table,\n database,\n uniq(table, partition) \"partitions\",\n count() AS parts,\n sum(bytes_on_disk) \"bytes_on_disk\",\n sum(data_uncompressed_bytes) as \"data_uncompressed_bytes\",\n sum(rows) as total_rows,\n max(col_count) as col_count\nFROM system.parts as parts\n LEFT JOIN (SELECT database, table, count() as col_count FROM system.columns GROUP BY table, database) as col_stats\n ON parts.table = col_stats.table AND col_stats.database = parts.database\nWHERE active\nGROUP BY table, database\n ) AS table_stats ON table.name = table_stats.table AND table.database = table_stats.database\nWHERE database IN (${database}) AND lower(name) != 'information_schema' AND table IN (${table})\nORDER BY bytes_on_disk DESC\nLIMIT 1000", + "refId": "A" + } + ], + "title": "Table summary", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": { + "col_count": 8, + "database": 0, + "disk_size": 6, + "engine": 2, + "name": 1, + "partitions": 4, + "parts": 5, + "table_stats.total_rows": 3, + "uncompressed_size": 7 + }, + "renameByName": { + "col_count": "Column count", + "col_stats.col_count": "Column count", + "database": "Database", + "disk_size": "Size on disk", + "engine": "Engine", + "name": "Table", + "partitions": "Partition count", + "parts": "Part count", + "table.database": "Database", + "table_stats.total_rows": "Row Count", + "tables": "Number of tables", + "total_rows": "Total rows", + "uncompressed_size": "Uncompressed size" + } + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Database" + }, + "properties": [ + { + "id": "custom.width", + "value": 94 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Table" + }, + "properties": [ + { + "id": "custom.width", + "value": 116 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Partition id" + }, + "properties": [ + { + "id": "custom.width", + "value": 103 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Disk" + }, + "properties": [ + { + "id": "custom.width", + "value": 104 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Reason" + }, + "properties": [ + { + "id": "custom.width", + "value": 125 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Min block number" + }, + "properties": [ + { + "id": "custom.width", + "value": 141 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Max block number" + }, + "properties": [ + { + "id": "custom.width", + "value": 139 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Level" + }, + "properties": [ + { + "id": "custom.width", + "value": 89 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Name" + }, + "properties": [ + { + "id": "custom.width", + "value": 168 + } + ] + } + ] + }, + "gridPos": { + "h": 10, + "w": 9, + "x": 15, + "y": 25 + }, + "id": 28, + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT database, table, partition_id, name, disk, level FROM system.detached_parts WHERE database IN (${database}) AND table IN (${table})", + "refId": "A" + } + ], + "title": "Detached partitions", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "database": "Database", + "disk": "Disk", + "level": "Level", + "max_block_number": "Max block number", + "min_block_number": "Min block number", + "name": "Name", + "partition_id": "Partition id", + "reason": "Reason", + "table": "Table" + } + } + } + ], + "type": "table" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "Rows in part", + "axisPlacement": "auto", + "axisWidth": 3, + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineStyle": { + "fill": "solid" + }, + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "always", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 11, + "w": 15, + "x": 0, + "y": 35 + }, + "id": 18, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT modification_time as timestamp, concatAssumeInjective(database, '.', table) as table, rows FROM system.parts parts WHERE parts.database IN ($database) AND parts.table IN (${table}) AND $__timeFilter(modification_time) ORDER BY modification_time ASC", + "refId": "A" + } + ], + "title": "Parts over time with row count", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": { + "bytes_on_disk": true, + "name": true, + "rows": false + }, + "indexByName": { + "rows": 2, + "table": 1, + "timestamp": 0 + }, + "renameByName": { + "rows": "rows in part" + } + } + }, + { + "id": "prepareTimeSeries", + "options": { + "format": "many" + } + } + ], + "type": "timeseries" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-GrYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "percentage", + "steps": [ + { + "color": "green" + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 11, + "w": 9, + "x": 15, + "y": 35 + }, + "id": 16, + "options": { + "displayMode": "gradient", + "minVizHeight": 10, + "minVizWidth": 0, + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": true + }, + "showUnfilled": true + }, + "pluginVersion": "9.0.1", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT concatAssumeInjective(database, '.', table) as dbTable, count() \"partitions\", sum(part_count) \"parts\", max(part_count) \"max_parts_per_partition\"\nFROM ( SELECT database, table, count() \"part_count\"\n FROM system.parts\n WHERE database IN (${database}) AND active AND table IN (${table})\n GROUP BY database, table, partition ) partitions\nGROUP BY database, table\nORDER BY max_parts_per_partition DESC\nLIMIT 10", + "refId": "A" + } + ], + "title": "Max parts per partition", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": { + "partitions": true, + "parts": true + }, + "indexByName": {}, + "renameByName": {} + } + } + ], + "type": "bargauge" + }, + { + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "auto", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Active" + }, + "properties": [ + { + "id": "custom.cellOptions", + "value": { + "mode": "basic", + "type": "color-background" + } + }, + { + "id": "mappings", + "value": [ + { + "options": { + "false": { + "color": "light-red", + "index": 1 + }, + "true": { + "color": "light-green", + "index": 0 + } + }, + "type": "value" + } + ] + }, + { + "id": "custom.width", + "value": 77 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "level" + }, + "properties": [ + { + "id": "custom.width", + "value": 69 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "path" + }, + "properties": [ + { + "id": "custom.width", + "value": 286 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Database" + }, + "properties": [ + { + "id": "custom.width", + "value": 88 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Table" + }, + "properties": [ + { + "id": "custom.width", + "value": 111 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Partition Name" + }, + "properties": [ + { + "id": "custom.width", + "value": 226 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "disk_name" + }, + "properties": [ + { + "id": "custom.width", + "value": 109 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "marks" + }, + "properties": [ + { + "id": "custom.width", + "value": 65 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "rows" + }, + "properties": [ + { + "id": "custom.width", + "value": 87 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "bytes_on_disk" + }, + "properties": [ + { + "id": "custom.width", + "value": 112 + } + ] + } + ] + }, + "gridPos": { + "h": 11, + "w": 24, + "x": 0, + "y": 46 + }, + "id": 20, + "options": { + "footer": { + "fields": "", + "reducer": [ + "sum" + ], + "show": false + }, + "showHeader": true, + "sortBy": [] + }, + "pluginVersion": "9.0.1", + "targets": [ + { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "format": 1, + "meta": { + "builderOptions": { + "fields": [], + "limit": 100, + "mode": "list" + } + }, + "queryType": "sql", + "rawSql": "SELECT database, table, partition_id, modification_time, name, part_type, active, level, disk_name, path, marks, rows, bytes_on_disk, refcount, min_block_number, max_block_number FROM system.parts WHERE database IN (${database}) AND table IN (${table}) AND modification_time > now() - INTERVAL 3 MINUTE ORDER BY modification_time DESC", + "refId": "A" + } + ], + "title": "Recent part analysis", + "transformations": [ + { + "id": "organize", + "options": { + "excludeByName": { + "is_frozen": true, + "marks": false, + "max_date": true, + "min_date": true, + "partition_id": true, + "refcount": true + }, + "indexByName": {}, + "renameByName": { + "active": "Active", + "database": "Database", + "engine": "Engine", + "name": "Partition Name", + "part_type": "Partition Type", + "partition_id": "Partition Id", + "table": "Table" + } + } + }, + { + "id": "convertFieldType", + "options": { + "conversions": [ + { + "destinationType": "boolean", + "targetField": "Active" + } + ], + "fields": {} + } + } + ], + "type": "table" + } + ], + "refresh": "", + "schemaVersion": 38, + "style": "dark", + "tags": [], + "templating": { + "list": [ + { + "current": { + "selected": false, + "text": "ClickHouse", + "value": "ClickHouse" + }, + "hide": 0, + "includeAll": false, + "label": "ClickHouse instance", + "multi": false, + "name": "datasource", + "options": [], + "query": "grafana-clickhouse-datasource", + "queryValue": "", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "type": "datasource" + }, + { + "current": { + "selected": false, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "definition": "SELECT name FROM system.databases;\n", + "hide": 0, + "includeAll": true, + "label": "Database", + "multi": true, + "name": "database", + "options": [], + "query": "SELECT name FROM system.databases;\n", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "type": "query" + }, + { + "current": { + "selected": false, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "grafana-clickhouse-datasource", + "uid": "${datasource}" + }, + "definition": "SELECT name FROM system.tables WHERE database IN (${database})", + "hide": 0, + "includeAll": true, + "label": "Table", + "multi": true, + "name": "table", + "options": [], + "query": "SELECT name FROM system.tables WHERE database IN (${database})", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "type": "query" + } + ] + }, + "time": { + "from": "now-24h", + "to": "now" + }, + "timepicker": {}, + "timezone": "", + "title": "ClickHouse - Data Analysis", + "uid": "-B3tt7a7z", + "version": 2, + "weekStart": "" +} \ No newline at end of file diff --git a/dev/local/grafana/datasources.yaml b/dev/local/grafana/datasources.yaml index 76cc3492..52ad61ea 100644 --- a/dev/local/grafana/datasources.yaml +++ b/dev/local/grafana/datasources.yaml @@ -37,3 +37,10 @@ datasources: url: http://prometheus:9090 uid: prometheus isDefault: true + + - name: ClickHouse + type: grafana-clickhouse-datasource + jsonData: + port: 9000 + server: clickhouse + tlsSkipVerify: true diff --git a/dev/local/otelcol.yml b/dev/local/otelcol.yml new file mode 100644 index 00000000..a2454a59 --- /dev/null +++ b/dev/local/otelcol.yml @@ -0,0 +1,60 @@ +receivers: + otlp: + protocols: + grpc: + http: + + prometheus: + config: + scrape_configs: + - job_name: otel-collector + scrape_interval: 5s + static_configs: + - targets: [localhost:8888] + - job_name: node-exporter + scrape_interval: 5s + static_configs: + - targets: [node-exporter:9100] + - job_name: prometheus + scrape_interval: 5s + static_configs: + - targets: [prometheus:9090] + - job_name: clickhouse + scrape_interval: 5s + static_configs: + - targets: [clickhouse:9090] +processors: + batch: + +exporters: + otlp: + endpoint: oteldb:4317 + tls: + insecure: true + prometheusremotewrite: + endpoint: "http://prometheus:9090/api/v1/write" + resource_to_telemetry_conversion: + enabled: true # Convert resource attributes to metric labels + tls: + insecure: true + +extensions: + health_check: + pprof: + zpages: + +service: + extensions: [health_check, pprof, zpages] + pipelines: + traces: + receivers: [otlp] + processors: [batch] + exporters: [otlp] + metrics: + receivers: [otlp, prometheus] + processors: [batch] + exporters: [otlp, prometheusremotewrite] + logs: + receivers: [otlp] + processors: [batch] + exporters: [otlp] diff --git a/dev/local/prometheus/prometheus.yml b/dev/local/prometheus/prometheus.yml index d7899212..99096624 100644 --- a/dev/local/prometheus/prometheus.yml +++ b/dev/local/prometheus/prometheus.yml @@ -4,26 +4,7 @@ global: evaluation_interval: 5s alerting: - alertmanagers: - - static_configs: - - targets: [] - scheme: http - timeout: 10s - api_version: v1 -scrape_configs: - - job_name: prometheus - honor_timestamps: true - metrics_path: /metrics - scheme: http - static_configs: - - targets: - - localhost:9090 - - - job_name: node-exporter - honor_timestamps: true - metrics_path: /metrics - scheme: http - static_configs: - - targets: - - node-exporter:9100 \ No newline at end of file +# scraping is done in otelcol +# metrics are ingested by prometheus remote write +scrape_configs: [] From 9d7e26ecf707e86a2d9991a32dbb6c5d4fb1cfdd Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 20 Nov 2023 14:20:27 +0300 Subject: [PATCH 004/112] fix(dev): fix clickhouse-data dashboard query --- dev/local/grafana/dashboards/clickhouse-data.json | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/local/grafana/dashboards/clickhouse-data.json b/dev/local/grafana/dashboards/clickhouse-data.json index 4887f1b0..f8e18ccd 100644 --- a/dev/local/grafana/dashboards/clickhouse-data.json +++ b/dev/local/grafana/dashboards/clickhouse-data.json @@ -24,7 +24,6 @@ "editable": true, "fiscalYearStartMonth": 0, "graphTooltip": 0, - "id": 4, "links": [], "liveNow": false, "panels": [ @@ -1085,8 +1084,9 @@ } }, "queryType": "sql", - "rawSql": "SELECT name,\n engine,\n tables,\n partitions,\n parts,\n formatReadableSize(bytes_on_disk) \"disk_size\",\n col_count,\n total_rows,\n formatReadableSize(data_uncompressed_bytes) as \"uncompressed_size\"\nFROM system.databases db\n LEFT JOIN ( SELECT database,\n uniq(table) \"tables\",\n uniq(table, partition) \"partitions\",\n count() AS parts,\n sum(bytes_on_disk) \"bytes_on_disk\",\n sum(data_compressed_bytes) as \"data_compressed_bytes\",\n sum(rows) as total_rows,\n max(col_count) as \"col_count\"\n FROM system.parts AS parts\n JOIN (SELECT database, count() as col_count\n FROM system.columns\n WHERE database IN (${database}) AND table IN (${table})\n GROUP BY database) as col_stats\n ON parts.database = col_stats.database\n WHERE database IN (${database}) AND active AND table IN (${table})\n GROUP BY database) AS db_stats ON db.name = db_stats.database\nWHERE database IN (${database}) AND lower(name) != 'information_schema'\nORDER BY bytes_on_disk DESC\nLIMIT 10;", - "refId": "A" + "rawSql": "SELECT name,\n engine,\n tables,\n partitions,\n parts,\n formatReadableSize(bytes_on_disk) \"disk_size\",\n col_count,\n total_rows,\n formatReadableSize(data_uncompressed_bytes) as \"uncompressed_size\"\nFROM system.databases db\n LEFT JOIN ( SELECT database,\n uniq(table) \"tables\",\n uniq(table, partition) \"partitions\",\n count() AS parts,\n sum(bytes_on_disk) \"bytes_on_disk\",\n sum(data_compressed_bytes) as \"data_compressed_bytes\",\n sum(data_uncompressed_bytes) as \"data_uncompressed_bytes\",\n sum(rows) as total_rows,\n max(col_count) as \"col_count\"\n FROM system.parts AS parts\n JOIN (SELECT database, count() as col_count\n FROM system.columns\n WHERE database IN (${database}) AND table IN (${table})\n GROUP BY database) as col_stats\n ON parts.database = col_stats.database\n WHERE database IN (${database}) AND active AND table IN (${table})\n GROUP BY database) AS db_stats ON db.name = db_stats.database\nWHERE database IN (${database}) AND lower(name) != 'information_schema'\nORDER BY bytes_on_disk DESC\nLIMIT 10;", + "refId": "A", + "selectedFormat": 4 } ], "title": "Database summary", @@ -2274,6 +2274,6 @@ "timezone": "", "title": "ClickHouse - Data Analysis", "uid": "-B3tt7a7z", - "version": 2, + "version": 1, "weekStart": "" } \ No newline at end of file From 00ac0e04ab7aa559807a95d4f0bd61b7edd34455 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Mon, 20 Nov 2023 15:50:30 +0300 Subject: [PATCH 005/112] refactor(chstorage): use JSON to store attributes --- internal/chstorage/attrs.go | 273 ----------------------- internal/chstorage/columns_metrics.go | 4 +- internal/chstorage/columns_traces.go | 298 +++++++++++--------------- internal/chstorage/querier_traces.go | 90 +++----- internal/chstorage/schema_traces.go | 196 ++++++++++++----- 5 files changed, 293 insertions(+), 568 deletions(-) delete mode 100644 internal/chstorage/attrs.go diff --git a/internal/chstorage/attrs.go b/internal/chstorage/attrs.go deleted file mode 100644 index 0c139c54..00000000 --- a/internal/chstorage/attrs.go +++ /dev/null @@ -1,273 +0,0 @@ -package chstorage - -import ( - "github.com/ClickHouse/ch-go/proto" - "go.opentelemetry.io/collector/pdata/pcommon" - - "github.com/go-faster/oteldb/internal/otelstorage" -) - -type chArrAttrCollector struct { - StrKeys [][]string - StrValues [][]string - IntKeys [][]string - IntValues [][]int64 - FloatKeys [][]string - FloatValues [][]float64 - BoolKeys [][]string - BoolValues [][]bool - BytesKeys [][]string - BytesValues [][]string -} - -func (c *chArrAttrCollector) Append(attrs otelstorage.Attrs) { - var ( - strKeys []string - strValues []string - intKeys []string - intValues []int64 - floatKeys []string - floatValues []float64 - boolKeys []string - boolValues []bool - bytesKeys []string - bytesValues []string - ) - - attrs.AsMap().Range(func(k string, v pcommon.Value) bool { - switch v.Type() { - case pcommon.ValueTypeStr: - strKeys = append(strKeys, k) - strValues = append(strValues, v.Str()) - case pcommon.ValueTypeBool: - boolKeys = append(boolKeys, k) - boolValues = append(boolValues, v.Bool()) - case pcommon.ValueTypeInt: - intKeys = append(intKeys, k) - intValues = append(intValues, v.Int()) - case pcommon.ValueTypeDouble: - floatKeys = append(floatKeys, k) - floatValues = append(floatValues, v.Double()) - case pcommon.ValueTypeBytes: - bytesKeys = append(bytesKeys, k) - bytesValues = append(bytesValues, string(v.Bytes().AsRaw())) - } - return true - }) - - c.StrKeys = append(c.StrKeys, strKeys) - c.StrValues = append(c.StrValues, strValues) - c.IntKeys = append(c.IntKeys, intKeys) - c.IntValues = append(c.IntValues, intValues) - c.FloatKeys = append(c.FloatKeys, floatKeys) - c.FloatValues = append(c.FloatValues, floatValues) - c.BoolKeys = append(c.BoolKeys, boolKeys) - c.BoolValues = append(c.BoolValues, boolValues) - c.BytesKeys = append(c.BytesKeys, bytesKeys) - c.BytesValues = append(c.BytesValues, bytesValues) -} - -func (c *chArrAttrCollector) AddRow(to *chArrAttrs) { - to.StrKeys.Append(c.StrKeys) - to.StrValues.Append(c.StrValues) - to.IntKeys.Append(c.IntKeys) - to.IntValues.Append(c.IntValues) - to.FloatKeys.Append(c.FloatKeys) - to.FloatValues.Append(c.FloatValues) - to.BoolKeys.Append(c.BoolKeys) - to.BoolValues.Append(c.BoolValues) - to.BytesKeys.Append(c.BytesKeys) - to.BytesValues.Append(c.BytesValues) -} - -type chArrAttrs struct { - StrKeys *proto.ColArr[[]string] - StrValues *proto.ColArr[[]string] - IntKeys *proto.ColArr[[]string] - IntValues *proto.ColArr[[]int64] - FloatKeys *proto.ColArr[[]string] - FloatValues *proto.ColArr[[]float64] - BoolKeys *proto.ColArr[[]string] - BoolValues *proto.ColArr[[]bool] - BytesKeys *proto.ColArr[[]string] - BytesValues *proto.ColArr[[]string] -} - -func arrayOfArray[T any, P proto.Arrayable[T]](e P) *proto.ColArr[[]T] { - return proto.NewArray[[]T](e.Array()) -} - -func newChArrAttrs() chArrAttrs { - return chArrAttrs{ - StrKeys: arrayOfArray[string](new(proto.ColStr).LowCardinality()), - StrValues: arrayOfArray[string](new(proto.ColStr)), - IntKeys: arrayOfArray[string](new(proto.ColStr).LowCardinality()), - IntValues: arrayOfArray[int64](new(proto.ColInt64)), - FloatKeys: arrayOfArray[string](new(proto.ColStr).LowCardinality()), - FloatValues: arrayOfArray[float64](new(proto.ColFloat64)), - BoolKeys: arrayOfArray[string](new(proto.ColStr).LowCardinality()), - BoolValues: arrayOfArray[bool](new(proto.ColBool)), - BytesKeys: arrayOfArray[string](new(proto.ColStr).LowCardinality()), - BytesValues: arrayOfArray[string](new(proto.ColStr)), - } -} - -func (c *chArrAttrs) Row(row int) (result []otelstorage.Attrs) { - var ( - rowStrKeys = c.StrKeys.Row(row) - rowStrValues = c.StrValues.Row(row) - rowIntKeys = c.IntKeys.Row(row) - rowIntValues = c.IntValues.Row(row) - rowFloatKeys = c.FloatKeys.Row(row) - rowFloatValues = c.FloatValues.Row(row) - rowBoolKeys = c.BoolKeys.Row(row) - rowBoolValues = c.BoolValues.Row(row) - rowBytesKeys = c.BytesKeys.Row(row) - rowBytesValues = c.BytesValues.Row(row) - ) - for i := range rowStrKeys { - var ( - strKeys = rowStrKeys[i] - strValues = rowStrValues[i] - intKeys = rowIntKeys[i] - intValues = rowIntValues[i] - floatKeys = rowFloatKeys[i] - floatValues = rowFloatValues[i] - boolKeys = rowBoolKeys[i] - boolValues = rowBoolValues[i] - bytesKeys = rowBytesKeys[i] - bytesValues = rowBytesValues[i] - ) - - m := pcommon.NewMap() - for i, key := range strKeys { - m.PutStr(key, strValues[i]) - } - for i, key := range intKeys { - m.PutInt(key, intValues[i]) - } - for i, key := range floatKeys { - m.PutDouble(key, floatValues[i]) - } - for i, key := range boolKeys { - m.PutBool(key, boolValues[i]) - } - for i, key := range bytesKeys { - data := m.PutEmptyBytes(key) - data.FromRaw([]byte(bytesValues[i])) - } - result = append(result, otelstorage.Attrs(m)) - } - return result -} - -type chAttrs struct { - StrKeys *proto.ColArr[string] - StrValues *proto.ColArr[string] - IntKeys *proto.ColArr[string] - IntValues *proto.ColArr[int64] - FloatKeys *proto.ColArr[string] - FloatValues *proto.ColArr[float64] - BoolKeys *proto.ColArr[string] - BoolValues *proto.ColArr[bool] - BytesKeys *proto.ColArr[string] - BytesValues *proto.ColArr[string] -} - -func newChAttrs() chAttrs { - return chAttrs{ - StrKeys: new(proto.ColStr).LowCardinality().Array(), - StrValues: new(proto.ColStr).Array(), - IntKeys: new(proto.ColStr).LowCardinality().Array(), - IntValues: new(proto.ColInt64).Array(), - FloatKeys: new(proto.ColStr).LowCardinality().Array(), - FloatValues: new(proto.ColFloat64).Array(), - BoolKeys: new(proto.ColStr).LowCardinality().Array(), - BoolValues: new(proto.ColBool).Array(), - BytesKeys: new(proto.ColStr).LowCardinality().Array(), - BytesValues: new(proto.ColStr).Array(), - } -} - -func (c *chAttrs) Append(attrs otelstorage.Attrs) { - var ( - strKeys []string - strValues []string - intKeys []string - intValues []int64 - floatKeys []string - floatValues []float64 - boolKeys []string - boolValues []bool - bytesKeys []string - bytesValues []string - ) - - attrs.AsMap().Range(func(k string, v pcommon.Value) bool { - switch v.Type() { - case pcommon.ValueTypeStr: - strKeys = append(strKeys, k) - strValues = append(strValues, v.Str()) - case pcommon.ValueTypeBool: - boolKeys = append(boolKeys, k) - boolValues = append(boolValues, v.Bool()) - case pcommon.ValueTypeInt: - intKeys = append(intKeys, k) - intValues = append(intValues, v.Int()) - case pcommon.ValueTypeDouble: - floatKeys = append(floatKeys, k) - floatValues = append(floatValues, v.Double()) - case pcommon.ValueTypeBytes: - bytesKeys = append(bytesKeys, k) - bytesValues = append(bytesValues, string(v.Bytes().AsRaw())) - } - return true - }) - - c.StrKeys.Append(strKeys) - c.StrValues.Append(strValues) - c.IntKeys.Append(intKeys) - c.IntValues.Append(intValues) - c.FloatKeys.Append(floatKeys) - c.FloatValues.Append(floatValues) - c.BoolKeys.Append(boolKeys) - c.BoolValues.Append(boolValues) - c.BytesKeys.Append(bytesKeys) - c.BytesValues.Append(bytesValues) -} - -func (c *chAttrs) Row(row int) otelstorage.Attrs { - m := pcommon.NewMap() - - var ( - strKeys = c.StrKeys.Row(row) - strValues = c.StrValues.Row(row) - intKeys = c.IntKeys.Row(row) - intValues = c.IntValues.Row(row) - floatKeys = c.FloatKeys.Row(row) - floatValues = c.FloatValues.Row(row) - boolKeys = c.BoolKeys.Row(row) - boolValues = c.BoolValues.Row(row) - bytesKeys = c.BytesKeys.Row(row) - bytesValues = c.BytesValues.Row(row) - ) - - for i, key := range strKeys { - m.PutStr(key, strValues[i]) - } - for i, key := range intKeys { - m.PutInt(key, intValues[i]) - } - for i, key := range floatKeys { - m.PutDouble(key, floatValues[i]) - } - for i, key := range boolKeys { - m.PutBool(key, boolValues[i]) - } - for i, key := range bytesKeys { - data := m.PutEmptyBytes(key) - data.FromRaw([]byte(bytesValues[i])) - } - - return otelstorage.Attrs(m) -} diff --git a/internal/chstorage/columns_metrics.go b/internal/chstorage/columns_metrics.go index c6d14310..bdb15a41 100644 --- a/internal/chstorage/columns_metrics.go +++ b/internal/chstorage/columns_metrics.go @@ -1,8 +1,6 @@ package chstorage -import ( - "github.com/ClickHouse/ch-go/proto" -) +import "github.com/ClickHouse/ch-go/proto" type metricColumns struct { name *proto.ColLowCardinality[string] diff --git a/internal/chstorage/columns_traces.go b/internal/chstorage/columns_traces.go index dcdf9031..7f114ed4 100644 --- a/internal/chstorage/columns_traces.go +++ b/internal/chstorage/columns_traces.go @@ -4,6 +4,7 @@ import ( "time" "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/errors" "github.com/google/uuid" "github.com/go-faster/oteldb/internal/otelstorage" @@ -19,16 +20,16 @@ type spanColumns struct { kind proto.ColEnum8 start *proto.ColDateTime64 end *proto.ColDateTime64 - spanAttrs chAttrs statusCode proto.ColInt32 statusMessage proto.ColStr - batchID proto.ColUUID - resourceAttrs chAttrs + batchID proto.ColUUID + attributes proto.ColStr + resource proto.ColStr - scopeName proto.ColStr - scopeVersion proto.ColStr - scopeAttrs chAttrs + scopeName proto.ColStr + scopeVersion proto.ColStr + scopeAttributes proto.ColStr events eventsColumns links linksColumns @@ -36,14 +37,11 @@ type spanColumns struct { func newSpanColumns() *spanColumns { return &spanColumns{ - name: new(proto.ColStr).LowCardinality(), - start: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), - end: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), - spanAttrs: newChAttrs(), - resourceAttrs: newChAttrs(), - scopeAttrs: newChAttrs(), - events: newEventsColumns(), - links: newLinksColumns(), + name: new(proto.ColStr).LowCardinality(), + start: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + end: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + events: newEventsColumns(), + links: newLinksColumns(), } } @@ -57,70 +55,25 @@ func (c *spanColumns) Input() proto.Input { {Name: "kind", Data: proto.Wrap(&c.kind, kindDDL)}, {Name: "start", Data: c.start}, {Name: "end", Data: c.end}, - {Name: "attrs_str_keys", Data: c.spanAttrs.StrKeys}, - {Name: "attrs_str_values", Data: c.spanAttrs.StrValues}, - {Name: "attrs_int_keys", Data: c.spanAttrs.IntKeys}, - {Name: "attrs_int_values", Data: c.spanAttrs.IntValues}, - {Name: "attrs_float_keys", Data: c.spanAttrs.FloatKeys}, - {Name: "attrs_float_values", Data: c.spanAttrs.FloatValues}, - {Name: "attrs_bool_keys", Data: c.spanAttrs.BoolKeys}, - {Name: "attrs_bool_values", Data: c.spanAttrs.BoolValues}, - {Name: "attrs_bytes_keys", Data: c.spanAttrs.BytesKeys}, - {Name: "attrs_bytes_values", Data: c.spanAttrs.BytesValues}, {Name: "status_code", Data: c.statusCode}, {Name: "status_message", Data: c.statusMessage}, {Name: "batch_id", Data: c.batchID}, - {Name: "resource_attrs_str_keys", Data: c.resourceAttrs.StrKeys}, - {Name: "resource_attrs_str_values", Data: c.resourceAttrs.StrValues}, - {Name: "resource_attrs_int_keys", Data: c.resourceAttrs.IntKeys}, - {Name: "resource_attrs_int_values", Data: c.resourceAttrs.IntValues}, - {Name: "resource_attrs_float_keys", Data: c.resourceAttrs.FloatKeys}, - {Name: "resource_attrs_float_values", Data: c.resourceAttrs.FloatValues}, - {Name: "resource_attrs_bool_keys", Data: c.resourceAttrs.BoolKeys}, - {Name: "resource_attrs_bool_values", Data: c.resourceAttrs.BoolValues}, - {Name: "resource_attrs_bytes_keys", Data: c.resourceAttrs.BytesKeys}, - {Name: "resource_attrs_bytes_values", Data: c.resourceAttrs.BytesValues}, + {Name: "attributes", Data: c.attributes}, + {Name: "resource", Data: c.resource}, {Name: "scope_name", Data: c.scopeName}, {Name: "scope_version", Data: c.scopeVersion}, - {Name: "scope_attrs_str_keys", Data: c.scopeAttrs.StrKeys}, - {Name: "scope_attrs_str_values", Data: c.scopeAttrs.StrValues}, - {Name: "scope_attrs_int_keys", Data: c.scopeAttrs.IntKeys}, - {Name: "scope_attrs_int_values", Data: c.scopeAttrs.IntValues}, - {Name: "scope_attrs_float_keys", Data: c.scopeAttrs.FloatKeys}, - {Name: "scope_attrs_float_values", Data: c.scopeAttrs.FloatValues}, - {Name: "scope_attrs_bool_keys", Data: c.scopeAttrs.BoolKeys}, - {Name: "scope_attrs_bool_values", Data: c.scopeAttrs.BoolValues}, - {Name: "scope_attrs_bytes_keys", Data: c.scopeAttrs.BytesKeys}, - {Name: "scope_attrs_bytes_values", Data: c.scopeAttrs.BytesValues}, - - {Name: "events_timestamps", Data: c.events.Timestamps}, - {Name: "events_names", Data: c.events.Names}, - {Name: "events_attrs_str_keys", Data: c.events.Attrs.StrKeys}, - {Name: "events_attrs_str_values", Data: c.events.Attrs.StrValues}, - {Name: "events_attrs_int_keys", Data: c.events.Attrs.IntKeys}, - {Name: "events_attrs_int_values", Data: c.events.Attrs.IntValues}, - {Name: "events_attrs_float_keys", Data: c.events.Attrs.FloatKeys}, - {Name: "events_attrs_float_values", Data: c.events.Attrs.FloatValues}, - {Name: "events_attrs_bool_keys", Data: c.events.Attrs.BoolKeys}, - {Name: "events_attrs_bool_values", Data: c.events.Attrs.BoolValues}, - {Name: "events_attrs_bytes_keys", Data: c.events.Attrs.BytesKeys}, - {Name: "events_attrs_bytes_values", Data: c.events.Attrs.BytesValues}, - - {Name: "links_trace_ids", Data: c.links.TraceIDs}, - {Name: "links_span_ids", Data: c.links.SpanIDs}, - {Name: "links_tracestates", Data: c.links.Tracestates}, - {Name: "links_attrs_str_keys", Data: c.links.Attrs.StrKeys}, - {Name: "links_attrs_str_values", Data: c.links.Attrs.StrValues}, - {Name: "links_attrs_int_keys", Data: c.links.Attrs.IntKeys}, - {Name: "links_attrs_int_values", Data: c.links.Attrs.IntValues}, - {Name: "links_attrs_float_keys", Data: c.links.Attrs.FloatKeys}, - {Name: "links_attrs_float_values", Data: c.links.Attrs.FloatValues}, - {Name: "links_attrs_bool_keys", Data: c.links.Attrs.BoolKeys}, - {Name: "links_attrs_bool_values", Data: c.links.Attrs.BoolValues}, - {Name: "links_attrs_bytes_keys", Data: c.links.Attrs.BytesKeys}, - {Name: "links_attrs_bytes_values", Data: c.links.Attrs.BytesValues}, + {Name: "scope_attributes", Data: c.scopeAttributes}, + + {Name: "events_timestamps", Data: c.events.timestamps}, + {Name: "events_names", Data: c.events.names}, + {Name: "events_attributes", Data: c.events.attributes}, + + {Name: "links_trace_ids", Data: c.links.traceIDs}, + {Name: "links_span_ids", Data: c.links.spanIDs}, + {Name: "links_tracestates", Data: c.links.tracestates}, + {Name: "links_attributes", Data: c.links.attributes}, } } @@ -134,70 +87,25 @@ func (c *spanColumns) Result() proto.Results { {Name: "kind", Data: &c.kind}, {Name: "start", Data: c.start}, {Name: "end", Data: c.end}, - {Name: "attrs_str_keys", Data: c.spanAttrs.StrKeys}, - {Name: "attrs_str_values", Data: c.spanAttrs.StrValues}, - {Name: "attrs_int_keys", Data: c.spanAttrs.IntKeys}, - {Name: "attrs_int_values", Data: c.spanAttrs.IntValues}, - {Name: "attrs_float_keys", Data: c.spanAttrs.FloatKeys}, - {Name: "attrs_float_values", Data: c.spanAttrs.FloatValues}, - {Name: "attrs_bool_keys", Data: c.spanAttrs.BoolKeys}, - {Name: "attrs_bool_values", Data: c.spanAttrs.BoolValues}, - {Name: "attrs_bytes_keys", Data: c.spanAttrs.BytesKeys}, - {Name: "attrs_bytes_values", Data: c.spanAttrs.BytesValues}, {Name: "status_code", Data: &c.statusCode}, {Name: "status_message", Data: &c.statusMessage}, {Name: "batch_id", Data: &c.batchID}, - {Name: "resource_attrs_str_keys", Data: c.resourceAttrs.StrKeys}, - {Name: "resource_attrs_str_values", Data: c.resourceAttrs.StrValues}, - {Name: "resource_attrs_int_keys", Data: c.resourceAttrs.IntKeys}, - {Name: "resource_attrs_int_values", Data: c.resourceAttrs.IntValues}, - {Name: "resource_attrs_float_keys", Data: c.resourceAttrs.FloatKeys}, - {Name: "resource_attrs_float_values", Data: c.resourceAttrs.FloatValues}, - {Name: "resource_attrs_bool_keys", Data: c.resourceAttrs.BoolKeys}, - {Name: "resource_attrs_bool_values", Data: c.resourceAttrs.BoolValues}, - {Name: "resource_attrs_bytes_keys", Data: c.resourceAttrs.BytesKeys}, - {Name: "resource_attrs_bytes_values", Data: c.resourceAttrs.BytesValues}, + {Name: "attributes", Data: &c.attributes}, + {Name: "resource", Data: &c.resource}, {Name: "scope_name", Data: &c.scopeName}, {Name: "scope_version", Data: &c.scopeVersion}, - {Name: "scope_attrs_str_keys", Data: c.scopeAttrs.StrKeys}, - {Name: "scope_attrs_str_values", Data: c.scopeAttrs.StrValues}, - {Name: "scope_attrs_int_keys", Data: c.scopeAttrs.IntKeys}, - {Name: "scope_attrs_int_values", Data: c.scopeAttrs.IntValues}, - {Name: "scope_attrs_float_keys", Data: c.scopeAttrs.FloatKeys}, - {Name: "scope_attrs_float_values", Data: c.scopeAttrs.FloatValues}, - {Name: "scope_attrs_bool_keys", Data: c.scopeAttrs.BoolKeys}, - {Name: "scope_attrs_bool_values", Data: c.scopeAttrs.BoolValues}, - {Name: "scope_attrs_bytes_keys", Data: c.scopeAttrs.BytesKeys}, - {Name: "scope_attrs_bytes_values", Data: c.scopeAttrs.BytesValues}, - - {Name: "events_timestamps", Data: c.events.Timestamps}, - {Name: "events_names", Data: c.events.Names}, - {Name: "events_attrs_str_keys", Data: c.events.Attrs.StrKeys}, - {Name: "events_attrs_str_values", Data: c.events.Attrs.StrValues}, - {Name: "events_attrs_int_keys", Data: c.events.Attrs.IntKeys}, - {Name: "events_attrs_int_values", Data: c.events.Attrs.IntValues}, - {Name: "events_attrs_float_keys", Data: c.events.Attrs.FloatKeys}, - {Name: "events_attrs_float_values", Data: c.events.Attrs.FloatValues}, - {Name: "events_attrs_bool_keys", Data: c.events.Attrs.BoolKeys}, - {Name: "events_attrs_bool_values", Data: c.events.Attrs.BoolValues}, - {Name: "events_attrs_bytes_keys", Data: c.events.Attrs.BytesKeys}, - {Name: "events_attrs_bytes_values", Data: c.events.Attrs.BytesValues}, - - {Name: "links_trace_ids", Data: c.links.TraceIDs}, - {Name: "links_span_ids", Data: c.links.SpanIDs}, - {Name: "links_tracestates", Data: c.links.Tracestates}, - {Name: "links_attrs_str_keys", Data: c.links.Attrs.StrKeys}, - {Name: "links_attrs_str_values", Data: c.links.Attrs.StrValues}, - {Name: "links_attrs_int_keys", Data: c.links.Attrs.IntKeys}, - {Name: "links_attrs_int_values", Data: c.links.Attrs.IntValues}, - {Name: "links_attrs_float_keys", Data: c.links.Attrs.FloatKeys}, - {Name: "links_attrs_float_values", Data: c.links.Attrs.FloatValues}, - {Name: "links_attrs_bool_keys", Data: c.links.Attrs.BoolKeys}, - {Name: "links_attrs_bool_values", Data: c.links.Attrs.BoolValues}, - {Name: "links_attrs_bytes_keys", Data: c.links.Attrs.BytesKeys}, - {Name: "links_attrs_bytes_values", Data: c.links.Attrs.BytesValues}, + {Name: "scope_attributes", Data: &c.scopeAttributes}, + + {Name: "events_timestamps", Data: c.events.timestamps}, + {Name: "events_names", Data: c.events.names}, + {Name: "events_attributes", Data: c.events.attributes}, + + {Name: "links_trace_ids", Data: c.links.traceIDs}, + {Name: "links_span_ids", Data: c.links.spanIDs}, + {Name: "links_tracestates", Data: c.links.tracestates}, + {Name: "links_attributes", Data: c.links.attributes}, } } @@ -210,21 +118,45 @@ func (c *spanColumns) AddRow(s tracestorage.Span) { c.kind.Append(proto.Enum8(s.Kind)) c.start.Append(time.Unix(0, int64(s.Start))) c.end.Append(time.Unix(0, int64(s.End))) - c.spanAttrs.Append(s.Attrs) c.statusCode.Append(s.StatusCode) c.statusMessage.Append(s.StatusMessage) + // FIXME(tdakkota): use UUID in Span. c.batchID.Append(uuid.MustParse(s.BatchID)) - c.resourceAttrs.Append(s.ResourceAttrs) + c.attributes.Append(encodeAttributes(s.Attrs)) + c.resource.Append(encodeAttributes(s.ResourceAttrs)) + c.scopeName.Append(s.ScopeName) c.scopeVersion.Append(s.ScopeVersion) - c.scopeAttrs.Append(s.ScopeAttrs) + c.scopeAttributes.Append(encodeAttributes(s.ScopeAttrs)) + c.events.AddRow(s.Events) c.links.AddRow(s.Links) } -func (c *spanColumns) ReadRowsTo(spans []tracestorage.Span) []tracestorage.Span { +func (c *spanColumns) ReadRowsTo(spans []tracestorage.Span) ([]tracestorage.Span, error) { for i := 0; i < c.traceID.Rows(); i++ { + attrs, err := decodeAttributes(c.attributes.Row(i)) + if err != nil { + return nil, errors.Wrap(err, "decode attributes") + } + resource, err := decodeAttributes(c.resource.Row(i)) + if err != nil { + return nil, errors.Wrap(err, "decode resource") + } + scopeAttrs, err := decodeAttributes(c.scopeAttributes.Row(i)) + if err != nil { + return nil, errors.Wrap(err, "decode scope attributes") + } + events, err := c.events.Row(i) + if err != nil { + return nil, errors.Wrap(err, "decode events") + } + links, err := c.links.Row(i) + if err != nil { + return nil, errors.Wrap(err, "decode links") + } + spans = append(spans, tracestorage.Span{ TraceID: otelstorage.TraceID(c.traceID.Row(i)), SpanID: otelstorage.SpanIDFromUint64(c.spanID.Row(i)), @@ -234,33 +166,33 @@ func (c *spanColumns) ReadRowsTo(spans []tracestorage.Span) []tracestorage.Span Kind: int32(c.kind.Row(i)), Start: otelstorage.NewTimestampFromTime(c.start.Row(i)), End: otelstorage.NewTimestampFromTime(c.end.Row(i)), - Attrs: c.spanAttrs.Row(i), + Attrs: attrs, StatusCode: c.statusCode.Row(i), StatusMessage: c.statusMessage.Row(i), BatchID: c.batchID.Row(i).String(), - ResourceAttrs: c.resourceAttrs.Row(i), + ResourceAttrs: resource, ScopeName: c.scopeName.Row(i), ScopeVersion: c.scopeVersion.Row(i), - ScopeAttrs: c.scopeAttrs.Row(i), - Events: c.events.Row(i), - Links: c.links.Row(i), + ScopeAttrs: scopeAttrs, + Events: events, + Links: links, }) } - return spans + return spans, nil } type eventsColumns struct { - Names *proto.ColArr[string] - Timestamps *proto.ColArr[time.Time] - Attrs chArrAttrs + names *proto.ColArr[string] + timestamps *proto.ColArr[time.Time] + attributes *proto.ColArr[string] } func newEventsColumns() eventsColumns { return eventsColumns{ - Names: new(proto.ColStr).Array(), - Timestamps: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano).Array(), - Attrs: newChArrAttrs(), + names: new(proto.ColStr).Array(), + timestamps: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano).Array(), + attributes: new(proto.ColStr).Array(), } } @@ -268,54 +200,59 @@ func (c *eventsColumns) AddRow(events []tracestorage.Event) { var ( names []string timestamps []time.Time - attrs chArrAttrCollector + attrs []string ) for _, e := range events { names = append(names, e.Name) timestamps = append(timestamps, time.Unix(0, int64(e.Timestamp))) - attrs.Append(e.Attrs) + attrs = append(attrs, encodeAttributes(e.Attrs)) } - c.Names.Append(names) - c.Timestamps.Append(timestamps) - attrs.AddRow(&c.Attrs) + c.names.Append(names) + c.timestamps.Append(timestamps) + c.attributes.Append(attrs) } -func (c *eventsColumns) Row(row int) (events []tracestorage.Event) { +func (c *eventsColumns) Row(row int) (events []tracestorage.Event, _ error) { var ( - names = c.Names.Row(row) - timestamps = c.Timestamps.Row(row) - attrs = c.Attrs.Row(row) + names = c.names.Row(row) + timestamps = c.timestamps.Row(row) + attributes = c.attributes.Row(row) l = min( len(names), len(timestamps), - len(attrs), + len(attributes), ) ) for i := 0; i < l; i++ { + attrs, err := decodeAttributes(attributes[i]) + if err != nil { + return nil, errors.Wrap(err, "decode attributes") + } + events = append(events, tracestorage.Event{ Name: names[i], Timestamp: otelstorage.NewTimestampFromTime(timestamps[i]), - Attrs: attrs[i], + Attrs: attrs, }) } - return events + return events, nil } type linksColumns struct { - TraceIDs *proto.ColArr[uuid.UUID] - SpanIDs *proto.ColArr[uint64] - Tracestates *proto.ColArr[string] - Attrs chArrAttrs + traceIDs *proto.ColArr[uuid.UUID] + spanIDs *proto.ColArr[uint64] + tracestates *proto.ColArr[string] + attributes *proto.ColArr[string] } func newLinksColumns() linksColumns { return linksColumns{ - TraceIDs: new(proto.ColUUID).Array(), - SpanIDs: new(proto.ColUInt64).Array(), - Tracestates: new(proto.ColStr).Array(), - Attrs: newChArrAttrs(), + traceIDs: new(proto.ColUUID).Array(), + spanIDs: new(proto.ColUInt64).Array(), + tracestates: new(proto.ColStr).Array(), + attributes: new(proto.ColStr).Array(), } } @@ -324,42 +261,47 @@ func (c *linksColumns) AddRow(links []tracestorage.Link) { traceIDs []uuid.UUID spanIDs []uint64 tracestates []string - attrs chArrAttrCollector + attributes []string ) for _, l := range links { traceIDs = append(traceIDs, uuid.UUID(l.TraceID)) spanIDs = append(spanIDs, l.SpanID.AsUint64()) tracestates = append(tracestates, l.TraceState) - attrs.Append(l.Attrs) + attributes = append(attributes, encodeAttributes(l.Attrs)) } - c.TraceIDs.Append(traceIDs) - c.SpanIDs.Append(spanIDs) - c.Tracestates.Append(tracestates) - attrs.AddRow(&c.Attrs) + c.traceIDs.Append(traceIDs) + c.spanIDs.Append(spanIDs) + c.tracestates.Append(tracestates) + c.attributes.Append(attributes) } -func (c *linksColumns) Row(row int) (links []tracestorage.Link) { +func (c *linksColumns) Row(row int) (links []tracestorage.Link, _ error) { var ( - traceIDs = c.TraceIDs.Row(row) - spanIDs = c.SpanIDs.Row(row) - tracestates = c.Tracestates.Row(row) - attrs = c.Attrs.Row(row) + traceIDs = c.traceIDs.Row(row) + spanIDs = c.spanIDs.Row(row) + tracestates = c.tracestates.Row(row) + attributes = c.attributes.Row(row) l = min( len(traceIDs), len(spanIDs), len(tracestates), - len(attrs), + len(attributes), ) ) for i := 0; i < l; i++ { + attrs, err := decodeAttributes(attributes[i]) + if err != nil { + return nil, errors.Wrap(err, "decode attributes") + } + links = append(links, tracestorage.Link{ TraceID: otelstorage.TraceID(traceIDs[i]), SpanID: otelstorage.SpanIDFromUint64(spanIDs[i]), TraceState: tracestates[i], - Attrs: attrs[i], + Attrs: attrs, }) } - return links + return links, nil } diff --git a/internal/chstorage/querier_traces.go b/internal/chstorage/querier_traces.go index 1346dc98..7729d5db 100644 --- a/internal/chstorage/querier_traces.go +++ b/internal/chstorage/querier_traces.go @@ -9,8 +9,10 @@ import ( "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/proto" "github.com/go-faster/errors" + "github.com/go-faster/sdk/zctx" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" + "go.uber.org/zap" "github.com/go-faster/oteldb/internal/iterators" "github.com/go-faster/oteldb/internal/otelstorage" @@ -51,29 +53,18 @@ func (q *Querier) SearchTags(ctx context.Context, tags map[string]string, opts t } query.WriteString(" AND (") - for i, prefix := range []string{ - "attrs", - "scope_attrs", - "resource_attrs", + for i, column := range []string{ + "attributes", + "scope_attributes", + "resource", } { if i != 0 { query.WriteString(" OR ") } - for i, column := range []string{ - "str", - "int", - "float", - "bool", - "bytes", - } { - if i != 0 { - query.WriteString(" OR ") - } - fmt.Fprintf(&query, - `toString( %[1]s_%[2]s_values[indexOf(%[1]s_%[2]s_keys, %[3]s)] ) = %[4]s`, - prefix, column, singleQuoted(key), singleQuoted(value), - ) - } + fmt.Fprintf(&query, + `JSONExtract(%s, %s, 'String') = %s`, + column, singleQuoted(key), singleQuoted(value), + ) query.WriteByte('\n') } query.WriteByte(')') @@ -228,6 +219,7 @@ func (q *Querier) SelectSpansets(ctx context.Context, params traceqlengine.Selec }() query := q.buildSpansetsQuery(span, params) + zctx.From(ctx).Debug("Query", zap.String("query", query)) iter, err := q.querySpans(ctx, query) if err != nil { @@ -294,25 +286,14 @@ func (q *Querier) buildSpansetsQuery(span trace.Span, params traceqlengine.Selec // Just query spans with this attribute. attr := matcher.Attribute query.WriteString("(\n") - for i, prefix := range getTraceQLAttributeColumns(attr) { + for i, column := range getTraceQLAttributeColumns(attr) { if i != 0 { query.WriteString(" OR ") } - for i, column := range []string{ - "str", - "int", - "float", - "bool", - "bytes", - } { - if i != 0 { - query.WriteString(" OR ") - } - fmt.Fprintf(&query, - `has(%s_%s_keys, %s)`, - prefix, column, singleQuoted(attr.Name), - ) - } + fmt.Fprintf(&query, + `JSONHas(%s, %s) = 1`, + column, singleQuoted(attr.Name), + ) query.WriteByte('\n') } query.WriteString("\n)") @@ -340,33 +321,33 @@ func (q *Querier) buildSpansetsQuery(span trace.Span, params traceqlengine.Selec continue } - var value, typeSuffix string + var value, typeName string switch s := matcher.Static; s.Type { case traceql.TypeString: value = singleQuoted(s.Str) - typeSuffix = "str" + typeName = "String" case traceql.TypeInt: value = strconv.FormatInt(s.AsInt(), 10) - typeSuffix = "int" + typeName = "Int64" case traceql.TypeNumber: value = strconv.FormatFloat(s.AsNumber(), 'f', -1, 64) - typeSuffix = "float" + typeName = "Float64" case traceql.TypeBool: if s.AsBool() { value = "true" } else { value = "false" } - typeSuffix = "bool" + typeName = "Boolean" case traceql.TypeDuration: value = strconv.FormatInt(s.AsDuration().Nanoseconds(), 10) - typeSuffix = "int" + typeName = "Int64" case traceql.TypeSpanStatus: value = strconv.Itoa(int(s.AsSpanStatus())) - typeSuffix = "int" + typeName = "Int64" case traceql.TypeSpanKind: value = strconv.Itoa(int(s.AsSpanKind())) - typeSuffix = "int" + typeName = "Int64" default: // Unsupported for now. dropped++ @@ -398,9 +379,8 @@ func (q *Querier) buildSpansetsQuery(span trace.Span, params traceqlengine.Selec if i != 0 { query.WriteString("\nOR ") } - fmt.Fprintf(&query, "%[1]s_%[2]s_values[indexOf(%[1]s_%[2]s_keys, %[3]s)] %[4]s %[5]s", - column, typeSuffix, - singleQuoted(attr.Name), + fmt.Fprintf(&query, "JSONExtract(%s, %s, %s) %s %s", + column, singleQuoted(attr.Name), singleQuoted(typeName), cmp, value, ) } @@ -429,18 +409,18 @@ func getTraceQLAttributeColumns(attr traceql.Attribute) []string { switch attr.Scope { case traceql.ScopeNone: return []string{ - "attrs", - "scope_attrs", - "resource_attrs", + "attributes", + "scope_attributes", + "resource", } case traceql.ScopeResource: return []string{ - "scope_attrs", - "resource_attrs", + "scope_attributes", + "resource", } case traceql.ScopeSpan: return []string{ - "attrs", + "attributes", } default: return nil @@ -454,9 +434,9 @@ func (q *Querier) querySpans(ctx context.Context, query string) (iterators.Itera if err := q.ch.Do(ctx, ch.Query{ Body: query, Result: c.Result(), - OnResult: func(ctx context.Context, block proto.Block) error { - r = c.ReadRowsTo(r) - return nil + OnResult: func(ctx context.Context, block proto.Block) (err error) { + r, err = c.ReadRowsTo(r) + return err }, }); err != nil { return nil, errors.Wrap(err, "query") diff --git a/internal/chstorage/schema_traces.go b/internal/chstorage/schema_traces.go index 832e366d..4676d764 100644 --- a/internal/chstorage/schema_traces.go +++ b/internal/chstorage/schema_traces.go @@ -1,5 +1,12 @@ package chstorage +import ( + "github.com/go-faster/jx" + "go.opentelemetry.io/collector/pdata/pcommon" + + "github.com/go-faster/oteldb/internal/otelstorage" +) + const ( spansSchema = `CREATE TABLE IF NOT EXISTS %s ( @@ -11,82 +18,153 @@ const ( kind Enum8(` + kindDDL + `), start DateTime64(9), end DateTime64(9), - attrs_str_keys Array(LowCardinality(String)), - attrs_str_values Array(String), - attrs_int_keys Array(LowCardinality(String)), - attrs_int_values Array(Int64), - attrs_float_keys Array(LowCardinality(String)), - attrs_float_values Array(Float64), - attrs_bool_keys Array(LowCardinality(String)), - attrs_bool_values Array(Bool), - attrs_bytes_keys Array(LowCardinality(String)), - attrs_bytes_values Array(String), status_code Int32, status_message String, batch_id UUID, - resource_attrs_str_keys Array(LowCardinality(String)), - resource_attrs_str_values Array(String), - resource_attrs_int_keys Array(LowCardinality(String)), - resource_attrs_int_values Array(Int64), - resource_attrs_float_keys Array(LowCardinality(String)), - resource_attrs_float_values Array(Float64), - resource_attrs_bool_keys Array(LowCardinality(String)), - resource_attrs_bool_values Array(Bool), - resource_attrs_bytes_keys Array(LowCardinality(String)), - resource_attrs_bytes_values Array(String), + attributes String, + resource String, scope_name String, scope_version String, - scope_attrs_str_keys Array(LowCardinality(String)), - scope_attrs_str_values Array(String), - scope_attrs_int_keys Array(LowCardinality(String)), - scope_attrs_int_values Array(Int64), - scope_attrs_float_keys Array(LowCardinality(String)), - scope_attrs_float_values Array(Float64), - scope_attrs_bool_keys Array(LowCardinality(String)), - scope_attrs_bool_values Array(Bool), - scope_attrs_bytes_keys Array(LowCardinality(String)), - scope_attrs_bytes_values Array(String), + scope_attributes String, events_timestamps Array(DateTime64(9)), events_names Array(String), - events_attrs_str_keys Array(Array(LowCardinality(String))), - events_attrs_str_values Array(Array(String)), - events_attrs_int_keys Array(Array(LowCardinality(String))), - events_attrs_int_values Array(Array(Int64)), - events_attrs_float_keys Array(Array(LowCardinality(String))), - events_attrs_float_values Array(Array(Float64)), - events_attrs_bool_keys Array(Array(LowCardinality(String))), - events_attrs_bool_values Array(Array(Bool)), - events_attrs_bytes_keys Array(Array(LowCardinality(String))), - events_attrs_bytes_values Array(Array(String)), + events_attributes Array(String), links_trace_ids Array(UUID), links_span_ids Array(UInt64), links_tracestates Array(String), - links_attrs_str_keys Array(Array(LowCardinality(String))), - links_attrs_str_values Array(Array(String)), - links_attrs_int_keys Array(Array(LowCardinality(String))), - links_attrs_int_values Array(Array(Int64)), - links_attrs_float_keys Array(Array(LowCardinality(String))), - links_attrs_float_values Array(Array(Float64)), - links_attrs_bool_keys Array(Array(LowCardinality(String))), - links_attrs_bool_values Array(Array(Bool)), - links_attrs_bytes_keys Array(Array(LowCardinality(String))), - links_attrs_bytes_values Array(Array(String)), + links_attributes Array(String) ) ENGINE = MergeTree() PRIMARY KEY (trace_id, span_id);` - kindDDL = `'KIND_UNSPECIFIED' = 0,'KIND_INTERNAL' = 1,'KIND_SERVER' = 2,'KIND_CLIENT' = 3,'KIND_PRODUCER' = 4,'KIND_CONSUMER' = 5` - + kindDDL = `'KIND_UNSPECIFIED' = 0,'KIND_INTERNAL' = 1,'KIND_SERVER' = 2,'KIND_CLIENT' = 3,'KIND_PRODUCER' = 4,'KIND_CONSUMER' = 5` tagsSchema = `CREATE TABLE IF NOT EXISTS %s -( - name LowCardinality(String), - value String, - value_type Enum8(` + valueTypeDDL + `) -) -ENGINE = MergeTree() -PRIMARY KEY (name);` + ( + name LowCardinality(String), + value String, + value_type Enum8(` + valueTypeDDL + `) + ) + ENGINE = MergeTree() + PRIMARY KEY (name);` valueTypeDDL = `'EMPTY' = 0,'STR' = 1,'INT' = 2,'DOUBLE' = 3,'BOOL' = 4,'MAP' = 5,'SLICE' = 6,'BYTES' = 7` ) + +func encodeAttributes(attrs otelstorage.Attrs) string { + e := jx.GetEncoder() + defer jx.PutEncoder(e) + + encodeMap(e, attrs.AsMap()) + return e.String() +} + +func encodeValue(e *jx.Encoder, v pcommon.Value) { + switch v.Type() { + case pcommon.ValueTypeStr: + e.Str(v.Str()) + case pcommon.ValueTypeInt: + e.Int64(v.Int()) + case pcommon.ValueTypeDouble: + e.Float64(v.Double()) + case pcommon.ValueTypeBool: + e.Bool(v.Bool()) + case pcommon.ValueTypeMap: + m := v.Map() + encodeMap(e, m) + case pcommon.ValueTypeSlice: + s := v.Slice() + e.ArrStart() + for i := 0; i < s.Len(); i++ { + encodeValue(e, s.At(i)) + } + e.ArrEnd() + case pcommon.ValueTypeBytes: + e.ByteStr(v.Bytes().AsRaw()) + default: + e.Null() + } +} + +func encodeMap(e *jx.Encoder, m pcommon.Map) { + if otelstorage.Attrs(m).IsZero() { + e.ObjEmpty() + return + } + e.ObjStart() + m.Range(func(k string, v pcommon.Value) bool { + e.FieldStart(k) + encodeValue(e, v) + return true + }) + e.ObjEnd() +} + +func decodeAttributes(s string) (otelstorage.Attrs, error) { + result := pcommon.NewMap() + err := decodeMap(jx.DecodeStr(s), result) + return otelstorage.Attrs(result), err +} + +func decodeValue(d *jx.Decoder, val pcommon.Value) error { + switch d.Next() { + case jx.String: + v, err := d.Str() + if err != nil { + return err + } + val.SetStr(v) + return nil + case jx.Number: + n, err := d.Num() + if err != nil { + return err + } + if n.IsInt() { + v, err := n.Int64() + if err != nil { + return err + } + val.SetInt(v) + } else { + v, err := n.Float64() + if err != nil { + return err + } + val.SetDouble(v) + } + return nil + case jx.Null: + if err := d.Null(); err != nil { + return err + } + // Do nothing, keep value empty. + return nil + case jx.Bool: + v, err := d.Bool() + if err != nil { + return err + } + val.SetBool(v) + return nil + case jx.Array: + s := val.SetEmptySlice() + return d.Arr(func(d *jx.Decoder) error { + rval := s.AppendEmpty() + return decodeValue(d, rval) + }) + case jx.Object: + m := val.SetEmptyMap() + return decodeMap(d, m) + default: + return d.Skip() + } +} + +func decodeMap(d *jx.Decoder, m pcommon.Map) error { + return d.Obj(func(d *jx.Decoder, key string) error { + rval := m.PutEmpty(key) + return decodeValue(d, rval) + }) +} From b4f7d554ec236b7ec894e861ff1e210cca549d69 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Mon, 20 Nov 2023 15:51:01 +0300 Subject: [PATCH 006/112] chore(chstorage): trace Prometheus querier --- internal/chstorage/querier_metrics.go | 34 ++++++++++++++++++++++----- 1 file changed, 28 insertions(+), 6 deletions(-) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index daf3c108..075d9552 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -15,6 +15,8 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/annotations" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/trace" ) var _ storage.Queryable = (*Querier)(nil) @@ -29,18 +31,23 @@ func (q *Querier) Querier(mint, maxt int64) (storage.Querier, error) { maxTime = time.UnixMilli(maxt) } return &promQuerier{ - ch: q.ch, mint: minTime, maxt: maxTime, + + ch: q.ch, + tables: q.tables, + tracer: q.tracer, }, nil } type promQuerier struct { - ch chClient - table Tables - mint time.Time maxt time.Time + + ch chClient + tables Tables + + tracer trace.Tracer } var _ storage.Querier = (*promQuerier)(nil) @@ -83,7 +90,8 @@ type seriesKey struct { resource string } -func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHints, matchers ...*labels.Matcher) (storage.SeriesSet, error) { +func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHints, matchers ...*labels.Matcher) (_ storage.SeriesSet, rerr error) { + table := p.tables.Points var ( start = p.mint end = p.maxt @@ -97,8 +105,22 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin } } + ctx, span := p.tracer.Start(ctx, "SelectSeries", + trace.WithAttributes( + attribute.Int64("chstorage.start_range", start.UnixNano()), + attribute.Int64("chstorage.end_range", end.UnixNano()), + attribute.String("chstorage.table", table), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + var query strings.Builder - fmt.Fprintf(&query, "SELECT * FROM %#[1]q WHERE true\n", p.table.Points) + fmt.Fprintf(&query, "SELECT * FROM %#[1]q WHERE true\n", table) if !start.IsZero() { fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(ts) >= %d\n", start.UnixNano()) } From 2a5ba39fefa66a21f836ca7c4b30841f6816f2e4 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 20 Nov 2023 19:03:54 +0300 Subject: [PATCH 007/112] feat(dev): add jaeger Also use otelcol as primary endpoint --- dev/local/ch/docker-compose.yml | 18 ++++++++-- dev/local/grafana/datasources.yaml | 15 ++++++++- dev/local/grafana/grafana.ini | 2 +- dev/local/otelcol.yml | 51 ++++++++++++++++++++++++++--- dev/local/prometheus/prometheus.yml | 5 +++ 5 files changed, 83 insertions(+), 8 deletions(-) diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index 98ed3031..a55c310c 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -7,7 +7,7 @@ services: clickhouse: image: clickhouse/clickhouse-server:23.4 ports: - - "9009:9009" +# - "9009:9009" - "8123:8123" - "9000:9000" volumes: @@ -25,7 +25,7 @@ services: - METRICS_ADDR=:3201 - OTEL_METRICS_EXPORTER=prometheus - OTEL_EXPORTER_OTLP_TRACES_PROTOCOL=grpc - - OTEL_EXPORTER_OTLP_TRACES_ENDPOINT=http://localhost:4317 + - OTEL_EXPORTER_OTLP_TRACES_ENDPOINT=http://otelcol:4317 - OTEL_RESOURCE_ATTRIBUTES=service.name=go-faster.oteldb ports: # Querier ports @@ -81,6 +81,17 @@ services: - prometheus - oteldb - node-exporter + - jaeger + environment: + # probably excessive + - OTEL_EXPORTER_OTLP_TRACES_PROTOCOL=grpc + - OTEL_EXPORTER_OTLP_TRACES_ENDPOINT=http://otelcol:4317 + command: + - '--config' + - '/etc/otelcol-contrib/config.yaml' + # https://github.com/open-telemetry/opentelemetry-collector/blob/main/docs/observability.md#how-we-expose-telemetry + - '--feature-gates=telemetry.useOtelForInternalMetrics' + - '--feature-gates=telemetry.useOtelWithSDKConfigurationForInternalTelemetry' # to produce some metrics node-exporter: @@ -99,6 +110,9 @@ services: - '--web.max-requests=40' - '--collector.filesystem.mount-points-exclude=^/(sys|proc|dev|host|etc)($$|/)' + jaeger: + image: jaegertracing/all-in-one:latest + grafana: image: "grafana/grafana:10.0.0" environment: diff --git a/dev/local/grafana/datasources.yaml b/dev/local/grafana/datasources.yaml index 52ad61ea..004d6035 100644 --- a/dev/local/grafana/datasources.yaml +++ b/dev/local/grafana/datasources.yaml @@ -23,7 +23,7 @@ datasources: serviceMap: datasourceUid: promoteldb - - name: Prometheus (oteldb) + - name: PromQL oteldb type: prometheus access: proxy orgId: 1 @@ -44,3 +44,16 @@ datasources: port: 9000 server: clickhouse tlsSkipVerify: true + + - name: Jaeger + type: jaeger + uid: jaeger + url: http://jaeger:16686 + access: proxy + jsonData: + nodeGraph: + enabled: true + traceQuery: + timeShiftEnabled: true + spanStartTimeShift: '1h' + spanEndTimeShift: '-1h' \ No newline at end of file diff --git a/dev/local/grafana/grafana.ini b/dev/local/grafana/grafana.ini index e1ed3389..169761be 100644 --- a/dev/local/grafana/grafana.ini +++ b/dev/local/grafana/grafana.ini @@ -1,2 +1,2 @@ [tracing.opentelemetry.otlp] -address = "oteldb:4317" +address = "otelcol:4317" diff --git a/dev/local/otelcol.yml b/dev/local/otelcol.yml index a2454a59..9651df48 100644 --- a/dev/local/otelcol.yml +++ b/dev/local/otelcol.yml @@ -1,9 +1,23 @@ + + receivers: otlp: protocols: grpc: http: + hostmetrics: + scrapers: + cpu: + disk: + filesystem: + load: + memory: + network: + process: + processes: + paging: + prometheus: config: scrape_configs: @@ -23,6 +37,10 @@ receivers: scrape_interval: 5s static_configs: - targets: [clickhouse:9090] + - job_name: oteldb + scrape_interval: 5s + static_configs: + - targets: [oteldb:9090] processors: batch: @@ -31,12 +49,20 @@ exporters: endpoint: oteldb:4317 tls: insecure: true + otlp/jaeger: + endpoint: jaeger:4317 + tls: + insecure: true prometheusremotewrite: endpoint: "http://prometheus:9090/api/v1/write" resource_to_telemetry_conversion: enabled: true # Convert resource attributes to metric labels tls: insecure: true + debug: + verbosity: detailed + sampling_initial: 5 + sampling_thereafter: 200 extensions: health_check: @@ -49,12 +75,29 @@ service: traces: receivers: [otlp] processors: [batch] - exporters: [otlp] + exporters: [otlp, otlp/jaeger] metrics: - receivers: [otlp, prometheus] + receivers: [otlp, prometheus, hostmetrics] processors: [batch] - exporters: [otlp, prometheusremotewrite] + exporters: [otlp, prometheusremotewrite, debug] logs: receivers: [otlp] processors: [batch] - exporters: [otlp] + exporters: [otlp, debug] + # https://github.com/open-telemetry/opentelemetry-collector/blob/main/docs/observability.md#how-we-expose-telemetry + telemetry: + metrics: + readers: + - periodic: + interval: 5000 + exporter: + otlp: + protocol: grpc/protobuf + endpoint: http://localhost:4317 + traces: + processors: + - batch: + exporter: + otlp: + protocol: grpc/protobuf + endpoint: http://localhost:4317 \ No newline at end of file diff --git a/dev/local/prometheus/prometheus.yml b/dev/local/prometheus/prometheus.yml index 99096624..f8c1e1cf 100644 --- a/dev/local/prometheus/prometheus.yml +++ b/dev/local/prometheus/prometheus.yml @@ -8,3 +8,8 @@ alerting: # scraping is done in otelcol # metrics are ingested by prometheus remote write scrape_configs: [] + +# this is experimental +tracing: + endpoint: otelcol:4317 + insecure: true From 0ccc11aade3c88968e7f8d9dffd49f792e0346c0 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 20 Nov 2023 19:29:08 +0300 Subject: [PATCH 008/112] feat(dev): add otelcol dashboard --- dev/local/grafana/dashboards/otelcol.json | 1863 +++++++++++++++++++++ dev/local/otelcol.yml | 3 +- 2 files changed, 1864 insertions(+), 2 deletions(-) create mode 100644 dev/local/grafana/dashboards/otelcol.json diff --git a/dev/local/grafana/dashboards/otelcol.json b/dev/local/grafana/dashboards/otelcol.json new file mode 100644 index 00000000..857f3b9f --- /dev/null +++ b/dev/local/grafana/dashboards/otelcol.json @@ -0,0 +1,1863 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": { + "type": "grafana", + "uid": "-- Grafana --" + }, + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "target": { + "limit": 100, + "matchAny": false, + "tags": [], + "type": "dashboard" + }, + "type": "dashboard" + } + ] + }, + "description": "otelcol metrics dashboard", + "editable": true, + "fiscalYearStartMonth": 0, + "gnetId": 18309, + "graphTooltip": 0, + "links": [], + "liveNow": false, + "panels": [ + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 8, + "panels": [], + "title": "Process", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Otel Collector Instance", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 0, + "y": 1 + }, + "id": 6, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "count(count(otelcol_process_cpu_seconds_total{service_instance_id=~\".*\"}) by (service_instance_id))", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "Instance", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "otelcol_process_cpu_seconds", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 3, + "y": 1 + }, + "id": 24, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg(rate(otelcol_process_cpu_seconds_total{}[$__rate_interval])*100) by (instance)", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "Cpu", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Memory Rss\navg(otelcol_process_memory_rss{}) by (instance)", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 3, + "x": 6, + "y": 1 + }, + "id": 38, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg(otelcol_process_memory_rss{}) by (instance)", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "Memory", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "", + "gridPos": { + "h": 4, + "w": 15, + "x": 9, + "y": 1 + }, + "id": 32, + "options": { + "code": { + "language": "plaintext", + "showLineNumbers": false, + "showMiniMap": false + }, + "content": "## Opentelemetry Collector Data Ingress/Egress\n\n`service_version:` ${service_version}\n\n`opentelemetry collector:` contrib\n\n", + "mode": "markdown" + }, + "pluginVersion": "10.0.0", + "type": "text" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 10, + "panels": [], + "title": "Trace Pipeline", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "(avg(sum by(job) (rate(otelcol_exporter_sent_spans_total{}[$__range]))) / avg(sum by(job) (rate(otelcol_receiver_accepted_spans_total{}[$__range])))) ", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "light-blue", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "super-light-orange", + "value": 0.4 + }, + { + "color": "dark-blue", + "value": 0.9 + }, + { + "color": "super-light-orange", + "value": 1.2 + }, + { + "color": "dark-red", + "value": 2.1 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [] + }, + "gridPos": { + "h": 19, + "w": 3, + "x": 0, + "y": 6 + }, + "id": 55, + "options": { + "orientation": "vertical", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": false + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "avg(sum by(job) (rate(otelcol_exporter_sent_spans_total{}[$__range])))", + "format": "time_series", + "hide": true, + "instant": false, + "legendFormat": "__auto", + "range": true, + "refId": "export" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg(sum by(job) (rate(otelcol_receiver_accepted_spans_total{}[$__range])))", + "format": "time_series", + "hide": true, + "legendFormat": "__auto", + "range": true, + "refId": "acc" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "(avg(sum by(job) (rate(otelcol_exporter_sent_spans_total{}[$__range]))) / avg(sum by(job) (rate(otelcol_receiver_accepted_spans_total{}[$__range])))) ", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "Export Ratio", + "transformations": [], + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "gridPos": { + "h": 11, + "w": 21, + "x": 3, + "y": 6 + }, + "id": 4, + "options": { + "nodes": { + "mainStatUnit": "flops" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "label_join(label_join(\n(rate(otelcol_receiver_accepted_spans_total{}[$__interval]))\n, \"id\", \"\", \"transport\", \"receiver\")\n, \"title\", \"\", \"transport\", \"receiver\")\n\nor\n\nlabel_replace(label_replace(\nsum by(service_name) (rate(otelcol_receiver_accepted_spans_total{}[$__interval]))\n, \"id\", \"processor\", \"dummynode\", \"\")\n, \"title\", \"processor\", \"dummynode\", \"\")\n\nor\nlabel_replace(label_replace(\n(rate(otelcol_processor_batch_batch_send_size_count{}[$__interval]))\n, \"id\", \"$0\", \"processor\", \".*\")\n, \"title\", \"$0\", \"processor\", \".*\")\n\nor\nlabel_replace(label_replace(\nsum by(exporter) (rate(otelcol_exporter_sent_spans_total{}[$__interval]))\n, \"id\", \"exporter\", \"dummynode\", \"\")\n, \"title\", \"exporter\", \"dummynode\", \"\")\n \nor\nlabel_replace(label_replace(\nsum by(exporter) (rate(otelcol_exporter_sent_spans_total{}[$__interval]))\n, \"id\", \"$0\", \"exporter\", \".*\")\n, \"title\", \"$0\", \"exporter\", \".*\")", + "format": "table", + "instant": true, + "legendFormat": "__auto", + "range": false, + "refId": "nodes" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "label_join(\nlabel_replace(label_join(\n(rate(otelcol_receiver_accepted_spans_total{}[$__interval]))\n\n ,\"source\",\"\",\"transport\",\"receiver\")\n,\"target\",\"processor\",\"\",\"\")\n,\"id\",\"-\",\"source\",\"target\")\n\n or\n\n label_join(\nlabel_replace(label_replace(\n (rate(otelcol_processor_batch_batch_send_size_count{}[$__interval]))\n ,\"source\",\"processor\",\"\",\"\")\n,\"target\",\"$0\",\"processor\",\".*\")\n,\"id\",\"-\",\"source\",\"target\")\n\nor\n label_join(\nlabel_replace(label_replace(\n (rate(otelcol_processor_batch_batch_send_size_count{}[$__interval]))\n ,\"source\",\"$0\",\"processor\",\".*\")\n,\"target\",\"exporter\",\"\",\"\")\n,\"id\",\"-\",\"source\",\"target\")\n\nor\n label_join(\nlabel_replace(label_replace(\n (rate(otelcol_exporter_sent_spans_total{}[$__interval]))\n ,\"source\",\"exporter\",\"\",\"\")\n,\"target\",\"$0\",\"exporter\",\".*\")\n,\"id\",\"-\",\"source\",\"target\")\n\n", + "format": "table", + "hide": false, + "instant": true, + "legendFormat": "__auto", + "range": false, + "refId": "edges" + } + ], + "transformations": [], + "type": "nodeGraph" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Spans Accepted by Receiver and Transport", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "noValue": "no data", + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 5, + "x": 3, + "y": 17 + }, + "id": 12, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_receiver_accepted_spans_total{}[$__rate_interval])) by (receiver,transport)", + "legendFormat": "{{receiver}}-{{transport}}", + "range": true, + "refId": "A" + } + ], + "title": "Accepted", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total Spans Accepted ", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 3, + "x": 8, + "y": 17 + }, + "id": 13, + "options": { + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_receiver_accepted_spans_total{}[$__rate_interval])) ", + "legendFormat": "{{receiver}}-{{transport}}", + "range": true, + "refId": "A" + } + ], + "title": "Total ", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total Batch Processed", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "#EAB839", + "value": 1 + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 5, + "x": 11, + "y": 17 + }, + "id": 15, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "text": {}, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_processor_batch_batch_send_size_sum{}[$__rate_interval])) by (processor)", + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Batch", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 3, + "x": 16, + "y": 17 + }, + "id": 14, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "sum(rate(otelcol_exporter_sent_spans_total{}[$__interval])) ", + "format": "time_series", + "instant": false, + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Total ", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Sent by Exporter", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 5, + "x": 19, + "y": 17 + }, + "id": 30, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "sum(rate(otelcol_exporter_sent_spans_total{}[$__rate_interval])) by (exporter)", + "format": "time_series", + "instant": false, + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Sent", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "noValue": "no data", + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 3, + "w": 5, + "x": 3, + "y": 22 + }, + "id": 17, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_receiver_refused_spans_total{}[$__rate_interval])) by (receiver,transport)", + "legendFormat": "{{receiver}}-{{transport}}", + "range": true, + "refId": "A" + } + ], + "title": "Refused", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total Spans Accepted ", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 3, + "w": 3, + "x": 8, + "y": 22 + }, + "id": 18, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_receiver_refused_spans_total{}[$__rate_interval])) ", + "legendFormat": "{{receiver}}-{{transport}}", + "range": true, + "refId": "A" + } + ], + "title": "Total ", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "otelcol_exporter_send_failed_spans", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 3, + "w": 3, + "x": 16, + "y": 22 + }, + "id": 19, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "sum(rate(otelcol_exporter_send_failed_spans_total{}[$__rate_interval])) ", + "format": "time_series", + "instant": false, + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Total ", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Sent by Exporter\notelcol_exporter_send_failed_spans", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 3, + "w": 5, + "x": 19, + "y": 22 + }, + "id": 20, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "sum(rate(otelcol_exporter_send_failed_spans_total{}[$__rate_interval])) by (exporter)", + "format": "time_series", + "instant": false, + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Failed", + "type": "stat" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 25 + }, + "id": 22, + "panels": [], + "title": "Metrics Pipeline", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "avg(sum by(job) (rate(otelcol_exporter_sent_metric_points_total{}[$__range]))) versus avg(sum by(job) (rate(otelcol_receiver_accepted_metric_points_total{}[$__range])))", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "light-blue", + "value": null + }, + { + "color": "semi-dark-red", + "value": 0 + }, + { + "color": "super-light-orange", + "value": 0.4 + }, + { + "color": "dark-blue", + "value": 0.9 + }, + { + "color": "super-light-orange", + "value": 1.2 + }, + { + "color": "dark-red", + "value": 2.1 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [] + }, + "gridPos": { + "h": 19, + "w": 3, + "x": 0, + "y": 26 + }, + "id": 54, + "options": { + "orientation": "vertical", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "/.*/", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": false + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "avg(sum by(job) (rate(otelcol_exporter_sent_metric_points_total{}[$__range])))", + "format": "time_series", + "hide": true, + "instant": false, + "legendFormat": "__auto", + "range": true, + "refId": "export" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg(sum by(job) (rate(otelcol_receiver_accepted_metric_points_total{}[$__range])))", + "format": "time_series", + "hide": true, + "legendFormat": "__auto", + "range": true, + "refId": "acc" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "( avg(sum by(job) (rate(otelcol_exporter_sent_metric_points_total{}[$__range]))) /avg(sum by(job) (rate(otelcol_receiver_accepted_metric_points_total{}[$__range]))))", + "hide": false, + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "Export Ratio", + "transformations": [ + { + "id": "calculateField", + "options": { + "alias": "percent", + "binary": { + "left": "avg(sum by(job) (rate(otelcol_exporter_sent_metric_points_total{}[3600s])))", + "operator": "/", + "reducer": "sum", + "right": "avg(sum by(job) (rate(otelcol_receiver_accepted_metric_points_total{}[3600s])))" + }, + "mode": "binary", + "reduce": { + "reducer": "sum" + } + } + }, + { + "id": "organize", + "options": { + "excludeByName": { + "(sum(rate(otelcol_exporter_sent_metric_points_total{exporter=\"prometheus\"}[1m0s])) )": true, + "Time": true, + "avg(sum by(job) (rate(otelcol_exporter_sent_metric_points_total{}[3600s])))": true, + "avg(sum by(job) (rate(otelcol_receiver_accepted_metric_points_total{}[3600s])))": true, + "{instance=\"otelcol:9464\", job=\"otel\"}": true + }, + "indexByName": {}, + "renameByName": { + "Time": "", + "percent": "Percent", + "{exporter=\"logging\", instance=\"otelcol:8888\", job=\"otel-collector\", service_instance_id=\"fbfa720a-ebf9-45c8-a79a-9d3b6021a663\", service_name=\"otelcol-contrib\", service_version=\"0.70.0\"}": "" + } + } + } + ], + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Metrics Signalling Pipelines", + "gridPos": { + "h": 11, + "w": 21, + "x": 3, + "y": 26 + }, + "id": 25, + "options": { + "nodes": { + "mainStatUnit": "flops" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "\nlabel_join(label_join(\n(rate(otelcol_receiver_accepted_metric_points_total{}[$__interval]))\n, \"id\", \"\", \"transport\", \"receiver\")\n, \"title\", \"\", \"transport\", \"receiver\")\n\nor\n\nlabel_replace(label_replace(\nsum by(service_name) (rate(otelcol_receiver_accepted_spans_total{}[$__interval]))\n, \"id\", \"processor\", \"dummynode\", \"\")\n, \"title\", \"processor\", \"dummynode\", \"\")\n\n\n\nor\nlabel_replace(label_replace(\n(rate(otelcol_processor_batch_batch_send_size_count{}[$__interval]))\n, \"id\", \"$0\", \"processor\", \".*\")\n, \"title\", \"$0\", \"processor\", \".*\")\n\n\n\n\n\nor\nlabel_replace(label_replace(\nsum (rate(otelcol_exporter_sent_metric_points_total{}[$__interval]))\n, \"id\", \"exporter\", \"dummynode\", \"\")\n, \"title\", \"exporter\", \"dummynode\", \"\")\n\nor\nlabel_replace(label_replace(\nsum by(exporter) (rate(otelcol_exporter_sent_metric_points_total{}[$__interval]))\n, \"id\", \"$0\", \"exporter\", \".*\")\n, \"title\", \"$0\", \"exporter\", \".*\")", + "format": "table", + "instant": true, + "legendFormat": "__auto", + "range": false, + "refId": "nodes" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "label_join(\nlabel_replace(label_join(\n(rate(otelcol_receiver_accepted_metric_points_total{}[$__interval]))\n\n,\"source\",\"\",\"transport\",\"receiver\")\n,\"target\",\"processor\",\"\",\"\")\n,\"id\",\"-\",\"source\",\"target\")\n\n\nor\n\nlabel_join(\nlabel_replace(label_replace(\n(rate(otelcol_processor_batch_batch_send_size_count{}[$__interval]))\n,\"source\",\"processor\",\"\",\"\")\n,\"target\",\"$0\",\"processor\",\".*\")\n,\"id\",\"-\",\"source\",\"target\")\n\n\n\n\n\nor\n\n\nlabel_join(\nlabel_replace(label_replace(\n(rate(otelcol_processor_batch_batch_send_size_count{}[$__interval]))\n,\"source\",\"$0\",\"processor\",\".*\")\n,\"target\",\"exporter\",\"\",\"\")\n,\"id\",\"-\",\"source\",\"target\")\n\nor\nlabel_join(\nlabel_replace(label_replace(\n(rate(otelcol_exporter_sent_metric_points_total{}[$__interval]))\n,\"source\",\"exporter\",\"\",\"\")\n,\"target\",\"$0\",\"exporter\",\".*\")\n,\"id\",\"-\",\"source\",\"target\")", + "format": "table", + "hide": false, + "instant": true, + "legendFormat": "__auto", + "range": false, + "refId": "edges" + } + ], + "transformations": [], + "type": "nodeGraph" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "otelcol_receiver_accepted_metric_points", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "noValue": "no data", + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 5, + "x": 3, + "y": 37 + }, + "id": 26, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_receiver_accepted_metric_points_total{}[$__rate_interval])) by (receiver,transport)", + "legendFormat": "{{receiver}}-{{transport}}", + "range": true, + "refId": "A" + } + ], + "title": "Accepted", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "otelcol_receiver_accepted_metric_points\nTotal Accepted ", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 3, + "x": 8, + "y": 37 + }, + "id": 27, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_receiver_accepted_metric_points_total{}[$__rate_interval])) ", + "legendFormat": "{{receiver}}-{{transport}}", + "range": true, + "refId": "A" + } + ], + "title": "Total ", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "#EAB839", + "value": 1 + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 5, + "x": 11, + "y": 37 + }, + "id": 28, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "text": {}, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_processor_batch_batch_send_size_sum{}[$__rate_interval])) by (processor)", + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Batch", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total Export ", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 3, + "x": 16, + "y": 37 + }, + "id": 29, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "sum(rate(otelcol_exporter_sent_metric_points_total{}[$__rate_interval])) ", + "format": "time_series", + "instant": false, + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Total ", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Sent by Exporter", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 5, + "w": 5, + "x": 19, + "y": 37 + }, + "id": 16, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "sum(rate(otelcol_exporter_sent_metric_points_total{}[$__rate_interval])) by (exporter) ", + "format": "time_series", + "instant": false, + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Sent", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "noValue": "no data", + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 3, + "w": 5, + "x": 3, + "y": 42 + }, + "id": 47, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_receiver_refused_metric_points_total{}[$__rate_interval])) by (receiver,transport)", + "legendFormat": "{{receiver}}-{{transport}}", + "range": true, + "refId": "A" + } + ], + "title": "Refused", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total Refused \nsum(rate(otelcol_receiver_refused_metric_points_total{}[$__rate_interval])) ", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 3, + "w": 3, + "x": 8, + "y": 42 + }, + "id": 48, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(otelcol_receiver_refused_metric_points_total{}[$__rate_interval])) ", + "legendFormat": "{{receiver}}-{{transport}}", + "range": true, + "refId": "A" + } + ], + "title": "Total ", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Total Failed Export ", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 3, + "w": 3, + "x": 16, + "y": 42 + }, + "id": 49, + "options": { + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "showThresholdLabels": false, + "showThresholdMarkers": true + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "sum(rate(otelcol_exporter_send_failed_metric_points_total{}[$__rate_interval])) ", + "format": "time_series", + "instant": false, + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Total", + "type": "gauge" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "description": "Sent by Exporter\notelcol_exporter_send_failed_spans", + "fieldConfig": { + "defaults": { + "color": { + "mode": "continuous-BlYlRd" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "text", + "value": null + } + ] + }, + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 3, + "w": 5, + "x": 19, + "y": 42 + }, + "id": 50, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "sum(rate(otelcol_exporter_send_failed_metric_points_total{}[$__rate_interval])) by (exporter)", + "format": "time_series", + "instant": false, + "legendFormat": "{{processor}}", + "range": true, + "refId": "A" + } + ], + "title": "Failed", + "type": "stat" + } + ], + "refresh": "", + "schemaVersion": 38, + "style": "dark", + "tags": [], + "templating": { + "list": [ + { + "current": { + "selected": false, + "text": "Prometheus", + "value": "Prometheus" + }, + "hide": 0, + "includeAll": false, + "multi": false, + "name": "DS_PROMETHEUS", + "options": [], + "query": "prometheus", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "type": "datasource" + }, + { + "allValue": ".*", + "current": { + "isNone": true, + "selected": false, + "text": "None", + "value": "" + }, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "definition": "query_result(sum(otelcol_process_uptime{}) by (service_version))\n", + "hide": 2, + "includeAll": false, + "label": "service_version", + "multi": true, + "name": "service_version", + "options": [], + "query": { + "query": "query_result(sum(otelcol_process_uptime{}) by (service_version))\n", + "refId": "StandardVariableQuery" + }, + "refresh": 1, + "regex": "/.*service_version=\"(.*)\".*/", + "skipUrlSync": false, + "sort": 0, + "type": "query" + } + ] + }, + "time": { + "from": "now-5m", + "to": "now" + }, + "timepicker": {}, + "timezone": "", + "title": "Opentelemetry Collector Data Flow", + "uid": "otelcol", + "version": 1, + "weekStart": "" +} \ No newline at end of file diff --git a/dev/local/otelcol.yml b/dev/local/otelcol.yml index 9651df48..555ea3b9 100644 --- a/dev/local/otelcol.yml +++ b/dev/local/otelcol.yml @@ -60,7 +60,6 @@ exporters: tls: insecure: true debug: - verbosity: detailed sampling_initial: 5 sampling_thereafter: 200 @@ -79,7 +78,7 @@ service: metrics: receivers: [otlp, prometheus, hostmetrics] processors: [batch] - exporters: [otlp, prometheusremotewrite, debug] + exporters: [otlp, prometheusremotewrite] logs: receivers: [otlp] processors: [batch] From f13df45ae9cf5655a9961c92e340d589b95e5a4b Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 20 Nov 2023 20:13:14 +0300 Subject: [PATCH 009/112] feat(dev): add clickhouse trace exporter --- .gitignore | 1 + chotel.Dockerfile | 17 ++ cmd/chotel/main.go | 260 ++++++++++++++++++++++++++++++ dev/local/ch/docker-compose.yml | 18 ++- go.mod | 101 ++++++------ integration/chotele2e/e2e_test.go | 171 ++++++++++++++++++++ integration/tempoe2e/ch_test.go | 2 +- internal/chtrace/chtrace.go | 89 ++++++++++ 8 files changed, 606 insertions(+), 53 deletions(-) create mode 100644 chotel.Dockerfile create mode 100644 cmd/chotel/main.go create mode 100644 integration/chotele2e/e2e_test.go create mode 100644 internal/chtrace/chtrace.go diff --git a/.gitignore b/.gitignore index c886934a..2f51748e 100644 --- a/.gitignore +++ b/.gitignore @@ -20,6 +20,7 @@ secret.yaml /oteldb /otelproxy /otelfaker +/chotel # goreleaser dist diff --git a/chotel.Dockerfile b/chotel.Dockerfile new file mode 100644 index 00000000..72d30e69 --- /dev/null +++ b/chotel.Dockerfile @@ -0,0 +1,17 @@ +FROM golang:latest as builder + +WORKDIR /app + +COPY go.mod go.sum ./ +RUN go mod download + +COPY . ./ +RUN CGO_ENABLED=0 GOOS=linux go build -o /app/chotel ./cmd/chotel + +FROM alpine:latest +RUN apk --no-cache add ca-certificates + +WORKDIR /app +COPY --from=builder /app/chotel /chotel + +ENTRYPOINT ["/chotel"] diff --git a/cmd/chotel/main.go b/cmd/chotel/main.go new file mode 100644 index 00000000..02c58bc5 --- /dev/null +++ b/cmd/chotel/main.go @@ -0,0 +1,260 @@ +// Binary chotel exports clichkouse traces to otel collector. +package main + +import ( + "context" + "fmt" + "strings" + "time" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/errors" + "github.com/go-faster/sdk/app" + "github.com/go-faster/sdk/zctx" + "go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/exporters/otlp/otlptrace" + "go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc" + "go.opentelemetry.io/otel/metric" + "go.opentelemetry.io/otel/sdk/resource" + tracesdk "go.opentelemetry.io/otel/sdk/trace" + "go.opentelemetry.io/otel/sdk/trace/tracetest" + semconv "go.opentelemetry.io/otel/semconv/v1.10.0" + "go.opentelemetry.io/otel/trace" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + + "github.com/go-faster/oteldb/internal/chtrace" +) + +func main() { + app.Run(func(ctx context.Context, lg *zap.Logger, m *app.Metrics) (err error) { + a, err := NewApp(lg, m) + if err != nil { + return errors.Wrap(err, "init") + } + return a.Run(ctx) + }) +} + +type App struct { + log *zap.Logger + metrics *app.Metrics + + clickHouseAddr string + otlpAddr string + + latest time.Time + + spansSaved metric.Int64Counter + traceExporter *otlptrace.Exporter +} + +const DDL = `CREATE TABLE IF NOT EXISTS opentelemetry_span_export +( + trace_id UUID, + span_id UInt64, + exported_at DATETIME +) + ENGINE = MergeTree + ORDER BY (toStartOfMinute(exported_at), trace_id, span_id) + TTL toStartOfMinute(exported_at) + INTERVAL 10 MINUTE +` + +func NewApp(logger *zap.Logger, metrics *app.Metrics) (*App, error) { + a := &App{ + log: logger, + metrics: metrics, + clickHouseAddr: "clickhouse:9000", + otlpAddr: "otelcol:4317", + } + { + meter := metrics.MeterProvider().Meter("chotel") + var err error + if a.spansSaved, err = meter.Int64Counter("chotel.spans.saved"); err != nil { + return nil, err + } + } + return a, nil +} + +func (a *App) Run(ctx context.Context) error { + ctx = zctx.Base(ctx, a.log) + if err := a.setup(ctx); err != nil { + return errors.Wrap(err, "setup") + } + + g, ctx := errgroup.WithContext(ctx) + g.Go(func() error { return a.runSender(ctx) }) + return g.Wait() +} + +func (a *App) setup(ctx context.Context) error { + a.log.Info("Setup") + ctx, cancel := context.WithTimeout(ctx, time.Second*10) + defer cancel() + + a.log.Info("Connecting to clickhouse") + db, err := ch.Dial(ctx, ch.Options{ + Address: a.clickHouseAddr, + Logger: a.log.Named("ch"), + + OpenTelemetryInstrumentation: true, + + MeterProvider: a.metrics.MeterProvider(), + TracerProvider: a.metrics.TracerProvider(), + }) + if err != nil { + return errors.Wrap(err, "clickhouse") + } + defer func() { + _ = db.Close() + }() + if err := db.Ping(ctx); err != nil { + return errors.Wrap(err, "clickhouse ping") + } + a.log.Info("Connected to clickhouse") + if err := db.Do(ctx, ch.Query{Body: DDL}); err != nil { + return errors.Wrap(err, "ensure db") + } + + conn, err := grpc.DialContext(ctx, a.otlpAddr, + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithStatsHandler(otelgrpc.NewClientHandler( + otelgrpc.WithTracerProvider(a.metrics.TracerProvider()), + otelgrpc.WithMeterProvider(a.metrics.MeterProvider()), + )), + grpc.WithBlock(), + ) + if err != nil { + return errors.Wrap(err, "dial otlp") + } + traceExporter, err := otlptracegrpc.New(ctx, otlptracegrpc.WithGRPCConn(conn)) + if err != nil { + return errors.Wrap(err, "setup trace exporter") + } + a.traceExporter = traceExporter + a.log.Info("Initialized") + return nil +} + +func (a *App) send(ctx context.Context, now time.Time) error { + ctx, span := a.metrics.TracerProvider().Tracer("chotel").Start(ctx, "Export spans") + defer span.End() + + db, err := ch.Dial(ctx, ch.Options{ + Logger: a.log.Named("logs"), + Address: a.clickHouseAddr, + Compression: ch.CompressionZSTD, + + OpenTelemetryInstrumentation: true, + + MeterProvider: a.metrics.MeterProvider(), + TracerProvider: a.metrics.TracerProvider(), + }) + if err != nil { + return errors.Wrap(err, "clickhouse") + } + defer func() { _ = db.Close() }() + + t := chtrace.NewTable() + q := fmt.Sprintf("SELECT %s FROM system.opentelemetry_span_log log ", strings.Join(t.Columns(), ", ")) + q += " ANTI JOIN opentelemetry_span_export ose ON log.trace_id = ose.trace_id AND log.span_id = ose.span_id" + if !a.latest.IsZero() { + q += fmt.Sprintf(" PREWHERE start_time_us > %d", a.latest.Add(time.Minute).UnixMilli()) + } + q += " ORDER BY log.start_time_us DESC LIMIT 10000" + zctx.From(ctx).Debug("Selecting spans", + zap.String("query", q), + zap.Time("time", a.latest), + ) + var ( + batch []tracesdk.ReadOnlySpan + exported struct { + TraceID proto.ColUUID + SpanID proto.ColUInt64 + ExportedAt proto.ColDateTime + } + ) + var latest time.Time + if err := db.Do(ctx, ch.Query{ + Body: q, + Result: t.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + exported.TraceID = append(exported.TraceID, t.TraceID...) + exported.SpanID = append(exported.SpanID, t.SpanID...) + for _, r := range t.Rows() { + exported.ExportedAt.Append(now) + stub := tracetest.SpanStub{ + Resource: resource.NewSchemaless( + semconv.ServiceNameKey.String("clickhouse"), + ), + Name: r.OperationName, + StartTime: r.StartTime, + EndTime: r.FinishTime, + SpanContext: trace.NewSpanContext(trace.SpanContextConfig{ + TraceID: r.TraceID, + SpanID: r.SpanID, + }), + Parent: trace.NewSpanContext(trace.SpanContextConfig{ + TraceID: r.TraceID, + SpanID: r.ParentSpanID, + }), + } + for k, v := range r.Attributes { + stub.Attributes = append(stub.Attributes, attribute.String(k, v)) + } + if latest.Before(stub.EndTime) { + latest = stub.EndTime + } + batch = append(batch, stub.Snapshot()) + } + return nil + }, + }); err != nil { + return errors.Wrap(err, "query") + } + if err := a.traceExporter.ExportSpans(ctx, batch); err != nil { + return errors.Wrap(err, "export") + } + if err := db.Do(ctx, ch.Query{ + Body: "INSERT INTO opentelemetry_span_export (trace_id, span_id, exported_at) VALUES", + Input: proto.Input{ + {Name: "trace_id", Data: exported.TraceID}, + {Name: "span_id", Data: exported.SpanID}, + {Name: "exported_at", Data: exported.ExportedAt}, + }, + }); err != nil { + return errors.Wrap(err, "insert") + } + a.latest = latest + zctx.From(ctx).Info("Exported", + zap.Int("count", len(exported.TraceID)), + zap.String("latest_time", a.latest.String()), + ) + return nil +} + +func (a *App) runSender(ctx context.Context) error { + ticker := time.NewTicker(time.Second * 5) + defer ticker.Stop() + + // First immediate tick. + if err := a.send(ctx, time.Now()); err != nil { + return errors.Wrap(err, "send") + } + + for { + select { + case <-ctx.Done(): + return ctx.Err() + case now := <-ticker.C: + if err := a.send(ctx, now); err != nil { + return errors.Wrap(err, "send") + } + } + } +} diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index a55c310c..e066fa66 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -5,7 +5,7 @@ volumes: services: clickhouse: - image: clickhouse/clickhouse-server:23.4 + image: clickhouse/clickhouse-server:23.10 ports: # - "9009:9009" - "8123:8123" @@ -13,6 +13,22 @@ services: volumes: - ./clickhouse.xml:/etc/clickhouse-server/config.d/monitoring.xml + # Exports traces from clickhouse internal table to otel. + chotel: + image: go-faster/oteldb/chotel + restart: always + build: + # TODO(ernado): use just image, this should not change often + context: ../../../ + dockerfile: chotel.Dockerfile + environment: + - OTEL_EXPORTER_OTLP_ENDPOINT=http://otelcol:4317 + - OTEL_EXPORTER_OTLP_PROTOCOL=grpc + - OTEL_EXPORTER_OTLP_INSECURE=true + - OTEL_RESOURCE_ATTRIBUTES=service.name=go-faster.oteldb.chotel + depends_on: + - clickhouse + oteldb: image: go-faster/oteldb build: diff --git a/go.mod b/go.mod index 2baf2b9c..9974b9f5 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module github.com/go-faster/oteldb go 1.21.0 require ( - github.com/ClickHouse/ch-go v0.60.0 + github.com/ClickHouse/ch-go v0.58.2 github.com/Masterminds/sprig/v3 v3.2.3 github.com/cenkalti/backoff/v4 v4.2.1 github.com/cespare/xxhash/v2 v2.2.0 @@ -12,13 +12,13 @@ require ( github.com/fatih/color v1.16.0 github.com/go-faster/errors v0.7.0 github.com/go-faster/jx v1.1.0 - github.com/go-faster/sdk v0.11.0 + github.com/go-faster/sdk v0.10.2 github.com/go-faster/tcpproxy v0.1.0 github.com/go-logfmt/logfmt v0.6.0 github.com/google/uuid v1.4.0 github.com/grafana/pyroscope-go v1.0.4 github.com/ogen-go/ogen v0.78.1-pre.0 - github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.90.0 + github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.89.0 github.com/opentracing/opentracing-go v1.2.0 github.com/prometheus/common v0.45.0 github.com/prometheus/prometheus v0.48.0 @@ -26,20 +26,22 @@ require ( github.com/stretchr/testify v1.8.4 github.com/testcontainers/testcontainers-go v0.26.0 github.com/zeebo/xxh3 v1.0.2 - go.opentelemetry.io/collector/component v0.90.0 - go.opentelemetry.io/collector/confmap v0.90.0 - go.opentelemetry.io/collector/consumer v0.90.0 - go.opentelemetry.io/collector/exporter v0.90.0 - go.opentelemetry.io/collector/extension v0.90.0 - go.opentelemetry.io/collector/otelcol v0.90.0 - go.opentelemetry.io/collector/pdata v1.0.0 - go.opentelemetry.io/collector/receiver v0.90.0 - go.opentelemetry.io/collector/receiver/otlpreceiver v0.90.0 + go.opentelemetry.io/collector/component v0.89.0 + go.opentelemetry.io/collector/confmap v0.89.0 + go.opentelemetry.io/collector/consumer v0.89.0 + go.opentelemetry.io/collector/exporter v0.89.0 + go.opentelemetry.io/collector/extension v0.89.0 + go.opentelemetry.io/collector/otelcol v0.89.0 + go.opentelemetry.io/collector/pdata v1.0.0-rcv0018 + go.opentelemetry.io/collector/receiver v0.89.0 + go.opentelemetry.io/collector/receiver/otlpreceiver v0.89.0 go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 go.opentelemetry.io/contrib/zpages v0.46.1 go.opentelemetry.io/otel v1.21.0 go.opentelemetry.io/otel/bridge/opentracing v1.21.0 + go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.20.0 + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.20.0 go.opentelemetry.io/otel/metric v1.21.0 go.opentelemetry.io/otel/sdk v1.21.0 go.opentelemetry.io/otel/sdk/metric v1.21.0 @@ -52,7 +54,7 @@ require ( go4.org/netipx v0.0.0-20230303233057-f1b76eb4bb35 golang.org/x/exp v0.0.0-20231006140011-7918f672742d golang.org/x/sync v0.5.0 - golang.org/x/tools v0.16.0 + golang.org/x/tools v0.15.0 google.golang.org/grpc v1.59.0 ) @@ -78,7 +80,7 @@ require ( github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/dennwc/varint v1.0.0 // indirect github.com/dlclark/regexp2 v1.10.0 // indirect - github.com/dmarkham/enumer v1.5.9 // indirect + github.com/dmarkham/enumer v1.5.8 // indirect github.com/docker/distribution v2.8.2+incompatible // indirect github.com/docker/go-connections v0.4.0 // indirect github.com/docker/go-units v0.5.0 // indirect @@ -110,7 +112,7 @@ require ( github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/jpillora/backoff v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.17.3 // indirect + github.com/klauspost/compress v1.17.2 // indirect github.com/klauspost/cpuid/v2 v2.0.9 // indirect github.com/knadh/koanf v1.5.0 // indirect github.com/knadh/koanf/v2 v2.0.1 // indirect @@ -159,38 +161,35 @@ require ( github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/collector v0.90.0 // indirect - go.opentelemetry.io/collector/config/configauth v0.90.0 // indirect - go.opentelemetry.io/collector/config/configcompression v0.90.0 // indirect - go.opentelemetry.io/collector/config/configgrpc v0.90.0 // indirect - go.opentelemetry.io/collector/config/confighttp v0.90.0 // indirect - go.opentelemetry.io/collector/config/confignet v0.90.0 // indirect - go.opentelemetry.io/collector/config/configopaque v0.90.0 // indirect - go.opentelemetry.io/collector/config/configtelemetry v0.90.0 // indirect - go.opentelemetry.io/collector/config/configtls v0.90.0 // indirect - go.opentelemetry.io/collector/config/internal v0.90.0 // indirect - go.opentelemetry.io/collector/connector v0.90.0 // indirect - go.opentelemetry.io/collector/extension/auth v0.90.0 // indirect - go.opentelemetry.io/collector/featuregate v1.0.0 // indirect - go.opentelemetry.io/collector/processor v0.90.0 // indirect - go.opentelemetry.io/collector/semconv v0.90.0 // indirect - go.opentelemetry.io/collector/service v0.90.0 // indirect - go.opentelemetry.io/contrib/config v0.1.1 // indirect - go.opentelemetry.io/contrib/instrumentation/runtime v0.46.1 // indirect - go.opentelemetry.io/contrib/propagators/autoprop v0.46.1 // indirect - go.opentelemetry.io/contrib/propagators/aws v1.21.1 // indirect - go.opentelemetry.io/contrib/propagators/b3 v1.21.1 // indirect - go.opentelemetry.io/contrib/propagators/jaeger v1.21.1 // indirect - go.opentelemetry.io/contrib/propagators/ot v1.21.1 // indirect - go.opentelemetry.io/otel/bridge/opencensus v0.44.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v0.44.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.44.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0 // indirect - go.opentelemetry.io/otel/exporters/prometheus v0.44.0 // indirect - go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v0.44.0 // indirect - go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.21.0 // indirect + go.opentelemetry.io/collector v0.89.0 // indirect + go.opentelemetry.io/collector/config/configauth v0.89.0 // indirect + go.opentelemetry.io/collector/config/configcompression v0.89.0 // indirect + go.opentelemetry.io/collector/config/configgrpc v0.89.0 // indirect + go.opentelemetry.io/collector/config/confighttp v0.89.0 // indirect + go.opentelemetry.io/collector/config/confignet v0.89.0 // indirect + go.opentelemetry.io/collector/config/configopaque v0.89.0 // indirect + go.opentelemetry.io/collector/config/configtelemetry v0.89.0 // indirect + go.opentelemetry.io/collector/config/configtls v0.89.0 // indirect + go.opentelemetry.io/collector/config/internal v0.89.0 // indirect + go.opentelemetry.io/collector/connector v0.89.0 // indirect + go.opentelemetry.io/collector/extension/auth v0.89.0 // indirect + go.opentelemetry.io/collector/featuregate v1.0.0-rcv0018 // indirect + go.opentelemetry.io/collector/processor v0.89.0 // indirect + go.opentelemetry.io/collector/semconv v0.89.0 // indirect + go.opentelemetry.io/collector/service v0.89.0 // indirect + go.opentelemetry.io/contrib/instrumentation/runtime v0.46.0 // indirect + go.opentelemetry.io/contrib/propagators/autoprop v0.46.0 // indirect + go.opentelemetry.io/contrib/propagators/aws v1.21.0 // indirect + go.opentelemetry.io/contrib/propagators/b3 v1.21.0 // indirect + go.opentelemetry.io/contrib/propagators/jaeger v1.21.0 // indirect + go.opentelemetry.io/contrib/propagators/ot v1.21.0 // indirect + go.opentelemetry.io/otel/bridge/opencensus v0.43.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v0.43.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.43.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.20.0 // indirect + go.opentelemetry.io/otel/exporters/prometheus v0.43.0 // indirect + go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v0.43.0 // indirect + go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.20.0 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/automaxprocs v1.5.3 // indirect go.uber.org/goleak v1.3.0 // indirect @@ -199,17 +198,17 @@ require ( go.ytsaurus.tech/library/go/ptr v0.0.1 // indirect go.ytsaurus.tech/library/go/x/xreflect v0.0.2 // indirect go.ytsaurus.tech/library/go/x/xruntime v0.0.3 // indirect - golang.org/x/crypto v0.16.0 // indirect + golang.org/x/crypto v0.15.0 // indirect golang.org/x/mod v0.14.0 // indirect - golang.org/x/net v0.19.0 // indirect + golang.org/x/net v0.18.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect - golang.org/x/sys v0.15.0 // indirect + golang.org/x/sys v0.14.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/gonum v0.14.0 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20231012201019-e917dd12ba7a // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231012201019-e917dd12ba7a // indirect google.golang.org/protobuf v1.31.0 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/integration/chotele2e/e2e_test.go b/integration/chotele2e/e2e_test.go new file mode 100644 index 00000000..8bd3320e --- /dev/null +++ b/integration/chotele2e/e2e_test.go @@ -0,0 +1,171 @@ +package chotele2e + +import ( + "context" + "fmt" + "math/rand" + "os" + "strings" + "sync" + "testing" + "time" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/proto" + "github.com/cenkalti/backoff/v4" + "github.com/go-faster/errors" + "github.com/stretchr/testify/require" + "github.com/testcontainers/testcontainers-go" + tracesdk "go.opentelemetry.io/otel/sdk/trace" + "go.opentelemetry.io/otel/sdk/trace/tracetest" + "go.opentelemetry.io/otel/trace" + "go.uber.org/zap" + + "github.com/go-faster/oteldb/internal/chtrace" +) + +type randomIDGenerator struct { + sync.Mutex + rand *rand.Rand +} + +// NewSpanID returns a non-zero span ID from a randomly-chosen sequence. +func (gen *randomIDGenerator) NewSpanID(ctx context.Context, traceID trace.TraceID) (sid trace.SpanID) { + gen.Lock() + defer gen.Unlock() + gen.rand.Read(sid[:]) + return sid +} + +// NewIDs returns a non-zero trace ID and a non-zero span ID from a +// randomly-chosen sequence. +func (gen *randomIDGenerator) NewIDs(ctx context.Context) (tid trace.TraceID, sid trace.SpanID) { + gen.Lock() + defer gen.Unlock() + gen.rand.Read(tid[:]) + gen.rand.Read(sid[:]) + return tid, sid +} + +func discardResult() proto.Result { + return (&proto.Results{}).Auto() +} + +func TestIntegrationTrace(t *testing.T) { + t.Parallel() + if os.Getenv("E2E") == "" { + t.Skip("Set E2E env to run") + } + ctx := context.Background() + + req := testcontainers.ContainerRequest{ + Name: "oteldb-chotel-clickhouse", + Image: "clickhouse/clickhouse-server:23.10", + ExposedPorts: []string{"8123/tcp", "9000/tcp"}, + } + chContainer, err := testcontainers.GenericContainer(ctx, testcontainers.GenericContainerRequest{ + ContainerRequest: req, + Started: true, + Logger: testcontainers.TestLogger(t), + Reuse: true, + }) + require.NoError(t, err, "container start") + + endpoint, err := chContainer.PortEndpoint(ctx, "9000", "") + require.NoError(t, err, "container endpoint") + + connectBackoff := backoff.NewExponentialBackOff() + connectBackoff.InitialInterval = 2 * time.Second + connectBackoff.MaxElapsedTime = time.Minute + + exporter := tracetest.NewInMemoryExporter() + randSource := rand.NewSource(15) + tp := tracesdk.NewTracerProvider( + // Using deterministic random ids. + tracesdk.WithIDGenerator(&randomIDGenerator{ + rand: rand.New(randSource), + }), + tracesdk.WithBatcher(exporter, + tracesdk.WithBatchTimeout(0), // instant + ), + ) + connOpt := ch.Options{ + Address: endpoint, + Logger: zap.NewNop(), + OpenTelemetryInstrumentation: true, + TracerProvider: tp, + Settings: []ch.Setting{ + { + Key: "send_logs_level", + Value: "trace", + Important: true, + }, + }, + } + conn, err := backoff.RetryWithData(func() (*ch.Client, error) { + c, err := ch.Dial(ctx, connOpt) + if err != nil { + return nil, errors.Wrap(err, "dial") + } + return c, nil + }, connectBackoff) + if err != nil { + t.Fatal(err) + } + + // Should record trace and spans. + var traceID trace.TraceID + require.NoError(t, conn.Do(ctx, ch.Query{ + Body: "SELECT 1", + Result: discardResult(), + OnLog: func(ctx context.Context, l ch.Log) error { + sc := trace.SpanContextFromContext(ctx) + traceID = sc.TraceID() + t.Logf("[%s-%s]: %s", sc.TraceID(), sc.SpanID(), l.Text) + return nil + }, + })) + + require.True(t, traceID.IsValid(), "trace id not registered") + t.Log("trace_id", traceID) + + // Force flushing. + require.NoError(t, tp.ForceFlush(ctx)) + spans := exporter.GetSpans() + require.NotEmpty(t, spans) + require.NoError(t, conn.Do(ctx, ch.Query{Body: "system flush logs"})) + + table := chtrace.NewTable() + var traces []chtrace.Trace + require.NoError(t, conn.Do(ctx, ch.Query{ + Body: fmt.Sprintf("SELECT %s FROM system.opentelemetry_span_log", strings.Join(table.Columns(), ", ")), + Result: table.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + traces = append(traces, table.Rows()...) + return nil + }, + })) + require.NotEmpty(t, traces) + + var gotTraces []chtrace.Trace + var foundQuery bool + for _, tt := range traces { + if tt.TraceID != traceID { + continue + } + t.Logf("%+v", tt) + require.True(t, tt.SpanID != [8]byte{}) + require.True(t, tt.ParentSpanID != [8]byte{}) + require.False(t, tt.StartTime.IsZero()) + require.False(t, tt.FinishTime.IsZero()) + require.Less(t, time.Since(tt.FinishTime), time.Hour) + require.NotEmpty(t, tt.OperationName) + if tt.OperationName == "query" { + require.Equal(t, tt.Attributes["db.statement"], "SELECT 1") + foundQuery = true + } + gotTraces = append(gotTraces, tt) + } + require.NotEmpty(t, gotTraces, "no traces found by trace_id") + require.True(t, foundQuery, "query span should be found") +} diff --git a/integration/tempoe2e/ch_test.go b/integration/tempoe2e/ch_test.go index 497a47c9..efcdcf38 100644 --- a/integration/tempoe2e/ch_test.go +++ b/integration/tempoe2e/ch_test.go @@ -27,7 +27,7 @@ func TestCH(t *testing.T) { req := testcontainers.ContainerRequest{ Name: "oteldb-tempoe2e-clickhouse", - Image: "clickhouse/clickhouse-server:23.4", + Image: "clickhouse/clickhouse-server:23.10", ExposedPorts: []string{"8123/tcp", "9000/tcp"}, } chContainer, err := testcontainers.GenericContainer(ctx, testcontainers.GenericContainerRequest{ diff --git a/internal/chtrace/chtrace.go b/internal/chtrace/chtrace.go new file mode 100644 index 00000000..925a15d2 --- /dev/null +++ b/internal/chtrace/chtrace.go @@ -0,0 +1,89 @@ +// Package chtrace implements reading ClickHouse traces. +package chtrace + +import ( + "encoding/binary" + "time" + + "github.com/ClickHouse/ch-go/proto" + "go.opentelemetry.io/otel/trace" +) + +// Table represents otel span table. +type Table struct { + TraceID proto.ColUUID // trace_id + SpanID proto.ColUInt64 // span_id + ParentSpanID proto.ColUInt64 // parent_span_id + OperationName *proto.ColLowCardinality[string] // operation_name + StartTimeMicro proto.ColUInt64 // start_time_us + FinishTimeMicro proto.ColUInt64 // finish_time_us + FinishDate proto.ColDate // finish_date + Attributes proto.ColMap[string, string] // attribute +} + +func (t Table) Rows() []Trace { + var out []Trace + for i := 0; i < t.TraceID.Rows(); i++ { + tt := Trace{ + TraceID: trace.TraceID(t.TraceID.Row(i)), + StartTime: time.UnixMicro(int64(t.StartTimeMicro.Row(i))), + FinishTime: time.UnixMicro(int64(t.FinishTimeMicro.Row(i))), + Attributes: t.Attributes.Row(i), + OperationName: t.OperationName.Row(i), + } + binary.BigEndian.PutUint64(tt.SpanID[:], t.SpanID.Row(i)) + binary.BigEndian.PutUint64(tt.ParentSpanID[:], t.ParentSpanID.Row(i)) + + out = append(out, tt) + } + + return out +} + +type Trace struct { + TraceID trace.TraceID + SpanID trace.SpanID + ParentSpanID trace.SpanID + OperationName string + StartTime time.Time + FinishTime time.Time + Attributes map[string]string +} + +func (t *Table) Result() proto.Results { + return proto.Results{ + {Name: "trace_id", Data: &t.TraceID}, + {Name: "span_id", Data: &t.SpanID}, + {Name: "parent_span_id", Data: &t.ParentSpanID}, + {Name: "operation_name", Data: t.OperationName}, + {Name: "start_time_us", Data: &t.StartTimeMicro}, + {Name: "finish_time_us", Data: &t.FinishTimeMicro}, + {Name: "finish_date", Data: &t.FinishDate}, + {Name: "attribute", Data: &t.Attributes}, + } +} + +func (t *Table) Columns() []string { + var out []string + for _, v := range t.Result() { + switch v.Name { + case "attribute_names": + out = append(out, "mapKeys(attribute) as attribute_names") + case "attribute_values": + out = append(out, "mapValues(attribute) as attribute_values") + default: + out = append(out, v.Name) + } + } + return out +} + +func NewTable() *Table { + return &Table{ + OperationName: new(proto.ColStr).LowCardinality(), + Attributes: proto.ColMap[string, string]{ + Keys: new(proto.ColStr).LowCardinality(), + Values: new(proto.ColStr), + }, + } +} From b9949c6beac7010dcd068ff4cc0c92bffe599321 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Mon, 20 Nov 2023 20:55:47 +0300 Subject: [PATCH 010/112] feat(chstorage): implement basic inserter --- internal/chstorage/columns_traces.go | 10 +-- internal/chstorage/inserter.go | 74 +------------------ internal/chstorage/inserter_metrics.go | 98 ++++++++++++++++++++++++++ internal/chstorage/inserter_traces.go | 79 +++++++++++++++++++++ internal/chstorage/schema.go | 5 ++ internal/chstorage/schema_metrics.go | 27 ++++--- internal/chstorage/schema_traces.go | 4 +- 7 files changed, 207 insertions(+), 90 deletions(-) create mode 100644 internal/chstorage/inserter_metrics.go create mode 100644 internal/chstorage/inserter_traces.go diff --git a/internal/chstorage/columns_traces.go b/internal/chstorage/columns_traces.go index 7f114ed4..b1840639 100644 --- a/internal/chstorage/columns_traces.go +++ b/internal/chstorage/columns_traces.go @@ -123,12 +123,12 @@ func (c *spanColumns) AddRow(s tracestorage.Span) { // FIXME(tdakkota): use UUID in Span. c.batchID.Append(uuid.MustParse(s.BatchID)) - c.attributes.Append(encodeAttributes(s.Attrs)) - c.resource.Append(encodeAttributes(s.ResourceAttrs)) + c.attributes.Append(encodeAttributes(s.Attrs.AsMap())) + c.resource.Append(encodeAttributes(s.ResourceAttrs.AsMap())) c.scopeName.Append(s.ScopeName) c.scopeVersion.Append(s.ScopeVersion) - c.scopeAttributes.Append(encodeAttributes(s.ScopeAttrs)) + c.scopeAttributes.Append(encodeAttributes(s.ScopeAttrs.AsMap())) c.events.AddRow(s.Events) c.links.AddRow(s.Links) @@ -205,7 +205,7 @@ func (c *eventsColumns) AddRow(events []tracestorage.Event) { for _, e := range events { names = append(names, e.Name) timestamps = append(timestamps, time.Unix(0, int64(e.Timestamp))) - attrs = append(attrs, encodeAttributes(e.Attrs)) + attrs = append(attrs, encodeAttributes(e.Attrs.AsMap())) } c.names.Append(names) @@ -267,7 +267,7 @@ func (c *linksColumns) AddRow(links []tracestorage.Link) { traceIDs = append(traceIDs, uuid.UUID(l.TraceID)) spanIDs = append(spanIDs, l.SpanID.AsUint64()) tracestates = append(tracestates, l.TraceState) - attributes = append(attributes, encodeAttributes(l.Attrs)) + attributes = append(attributes, encodeAttributes(l.Attrs.AsMap())) } c.traceIDs.Append(traceIDs) diff --git a/internal/chstorage/inserter.go b/internal/chstorage/inserter.go index 26de21b4..3f379a58 100644 --- a/internal/chstorage/inserter.go +++ b/internal/chstorage/inserter.go @@ -1,13 +1,8 @@ package chstorage import ( - "context" - - "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/chpool" - "github.com/ClickHouse/ch-go/proto" "go.opentelemetry.io/otel" - "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/trace" @@ -70,73 +65,6 @@ func NewInserter(c *chpool.Pool, opts InserterOptions) (*Inserter, error) { tables: opts.Tables, insertedSpans: insertedSpans, insertedTags: insertedTags, - tracer: opts.TracerProvider.Tracer("Spans.Inserter"), + tracer: opts.TracerProvider.Tracer("chstorage.Inserter"), }, nil } - -// InsertSpans inserts given spans. -func (i *Inserter) InsertSpans(ctx context.Context, spans []tracestorage.Span) (rerr error) { - table := i.tables.Spans - ctx, span := i.tracer.Start(ctx, "InsertSpans", trace.WithAttributes( - attribute.Int("chstorage.spans_count", len(spans)), - attribute.String("chstorage.table", table), - )) - defer func() { - if rerr != nil { - span.RecordError(rerr) - } else { - i.insertedSpans.Add(ctx, int64(len(spans))) - } - span.End() - }() - - c := newSpanColumns() - for _, s := range spans { - c.AddRow(s) - } - input := c.Input() - return i.ch.Do(ctx, ch.Query{ - Body: input.Into(table), - Input: input, - }) -} - -// InsertTags insert given set of tags to the storage. -func (i *Inserter) InsertTags(ctx context.Context, tags map[tracestorage.Tag]struct{}) (rerr error) { - table := i.tables.Tags - ctx, span := i.tracer.Start(ctx, "InsertTags", trace.WithAttributes( - attribute.Int("chstorage.tags_count", len(tags)), - attribute.String("chstorage.table", table), - )) - defer func() { - if rerr != nil { - span.RecordError(rerr) - } else { - i.insertedTags.Add(ctx, int64(len(tags))) - } - span.End() - }() - - var ( - name = new(proto.ColStr).LowCardinality() - value proto.ColStr - valueType proto.ColEnum8 - ) - - for tag := range tags { - name.Append(tag.Name) - value.Append(tag.Value) - valueType.Append(proto.Enum8(tag.Type)) - } - - input := proto.Input{ - {Name: "name", Data: name}, - {Name: "value", Data: value}, - {Name: "value_type", Data: proto.Wrap(&valueType, valueTypeDDL)}, - } - - return i.ch.Do(ctx, ch.Query{ - Body: input.Into(table), - Input: input, - }) -} diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go new file mode 100644 index 00000000..605df658 --- /dev/null +++ b/internal/chstorage/inserter_metrics.go @@ -0,0 +1,98 @@ +package chstorage + +import ( + "context" + + "github.com/ClickHouse/ch-go" + "github.com/go-faster/errors" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +// ConsumeMetrics inserts given metrics. +func (i *Inserter) ConsumeMetrics(ctx context.Context, metrics pmetric.Metrics) error { + c := newMetricColumns() + if err := i.mapMetrics(c, metrics); err != nil { + return errors.Wrap(err, "map metrics") + } + + input := c.Input() + if err := i.ch.Do(ctx, ch.Query{ + Body: input.Into(i.tables.Points), + Input: input, + }); err != nil { + return errors.Wrap(err, "insert") + } + + return nil +} + +func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics) error { + var ( + addPoints = func( + name string, + res pcommon.Map, + slices pmetric.NumberDataPointSlice, + ) error { + for i := 0; i < slices.Len(); i++ { + point := slices.At(i) + ts := point.Timestamp().AsTime() + attrs := point.Attributes() + + var val float64 + switch typ := point.ValueType(); typ { + case pmetric.NumberDataPointValueTypeInt: + // TODO(tdakkota): check for overflow + val = float64(point.IntValue()) + case pmetric.NumberDataPointValueTypeDouble: + val = point.DoubleValue() + default: + return errors.Errorf("unexpected metric %q value type: %v", name, typ) + } + + c.name.Append(name) + c.ts.Append(ts) + c.value.Append(val) + c.attributes.Append(encodeAttributes(attrs)) + c.resource.Append(encodeAttributes(res)) + } + return nil + } + + resMetrics = metrics.ResourceMetrics() + ) + for i := 0; i < resMetrics.Len(); i++ { + resMetric := resMetrics.At(i) + resAttrs := resMetric.Resource().Attributes() + + scopeMetrics := resMetric.ScopeMetrics() + for i := 0; i < scopeMetrics.Len(); i++ { + scopeLog := scopeMetrics.At(i) + + records := scopeLog.Metrics() + for i := 0; i < records.Len(); i++ { + record := records.At(i) + name := record.Name() + + switch typ := record.Type(); typ { + case pmetric.MetricTypeGauge: + gauge := record.Gauge() + if err := addPoints(name, resAttrs, gauge.DataPoints()); err != nil { + return err + } + case pmetric.MetricTypeSum: + sum := record.Sum() + if err := addPoints(name, resAttrs, sum.DataPoints()); err != nil { + return err + } + case pmetric.MetricTypeHistogram, pmetric.MetricTypeExponentialHistogram, pmetric.MetricTypeSummary: + // FIXME(tdakkota): ignore for now. + default: + return errors.Errorf("unexpected metric %q type %v", name, typ) + } + } + } + } + + return nil +} diff --git a/internal/chstorage/inserter_traces.go b/internal/chstorage/inserter_traces.go new file mode 100644 index 00000000..2589c2e4 --- /dev/null +++ b/internal/chstorage/inserter_traces.go @@ -0,0 +1,79 @@ +package chstorage + +import ( + "context" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/proto" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/trace" + + "github.com/go-faster/oteldb/internal/tracestorage" +) + +// InsertSpans inserts given spans. +func (i *Inserter) InsertSpans(ctx context.Context, spans []tracestorage.Span) (rerr error) { + table := i.tables.Spans + ctx, span := i.tracer.Start(ctx, "InsertSpans", trace.WithAttributes( + attribute.Int("chstorage.spans_count", len(spans)), + attribute.String("chstorage.table", table), + )) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } else { + i.insertedSpans.Add(ctx, int64(len(spans))) + } + span.End() + }() + + c := newSpanColumns() + for _, s := range spans { + c.AddRow(s) + } + input := c.Input() + return i.ch.Do(ctx, ch.Query{ + Body: input.Into(table), + Input: input, + }) +} + +// InsertTags insert given set of tags to the storage. +func (i *Inserter) InsertTags(ctx context.Context, tags map[tracestorage.Tag]struct{}) (rerr error) { + table := i.tables.Tags + ctx, span := i.tracer.Start(ctx, "InsertTags", trace.WithAttributes( + attribute.Int("chstorage.tags_count", len(tags)), + attribute.String("chstorage.table", table), + )) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } else { + i.insertedTags.Add(ctx, int64(len(tags))) + } + span.End() + }() + + var ( + name = new(proto.ColStr).LowCardinality() + value proto.ColStr + valueType proto.ColEnum8 + ) + + for tag := range tags { + name.Append(tag.Name) + value.Append(tag.Value) + valueType.Append(proto.Enum8(tag.Type)) + } + + input := proto.Input{ + {Name: "name", Data: name}, + {Name: "value", Data: value}, + {Name: "value_type", Data: proto.Wrap(&valueType, valueTypeDDL)}, + } + + return i.ch.Do(ctx, ch.Query{ + Body: input.Into(table), + Input: input, + }) +} diff --git a/internal/chstorage/schema.go b/internal/chstorage/schema.go index 5b0ab708..4f9a0fff 100644 --- a/internal/chstorage/schema.go +++ b/internal/chstorage/schema.go @@ -14,6 +14,7 @@ type Tables struct { Tags string Points string + Labels string } var defaultTables = Tables{ @@ -21,6 +22,7 @@ var defaultTables = Tables{ Tags: "traces_tags", Points: "metrics_points", + Labels: "metrics_labels", } type chClient interface { @@ -36,6 +38,9 @@ func (t Tables) Create(ctx context.Context, c chClient) error { for _, s := range []schema{ {t.Spans, spansSchema}, {t.Tags, tagsSchema}, + + {t.Points, pointsSchema}, + {t.Labels, labelsSchema}, } { if err := c.Do(ctx, ch.Query{ Body: fmt.Sprintf(s.query, s.name), diff --git a/internal/chstorage/schema_metrics.go b/internal/chstorage/schema_metrics.go index c56da942..1ab4aa0f 100644 --- a/internal/chstorage/schema_metrics.go +++ b/internal/chstorage/schema_metrics.go @@ -8,16 +8,23 @@ import ( ) const ( - metricSchema = `CREATE TABLE IF NOT EXISTS %s -( - name LowCardinality(String), - ts DateTime64(9), - value Float64, - attributes String, - resource String -) -ENGINE = MergeTree() -ORDER BY ts;` + pointsSchema = `CREATE TABLE IF NOT EXISTS %s + ( + name LowCardinality(String), + timestamp DateTime64(9), + value Float64, + attributes String, + resource String + ) + ENGINE = MergeTree() + ORDER BY ts;` + labelsSchema = `CREATE TABLE IF NOT EXISTS %s + ( + name LowCardinality(String), + value String + ) + ENGINE = MergeTree() + PRIMARY KEY (name);` ) func parseLabels(s string, to map[string]string) error { diff --git a/internal/chstorage/schema_traces.go b/internal/chstorage/schema_traces.go index 4676d764..1f3fbeda 100644 --- a/internal/chstorage/schema_traces.go +++ b/internal/chstorage/schema_traces.go @@ -52,11 +52,11 @@ PRIMARY KEY (trace_id, span_id);` valueTypeDDL = `'EMPTY' = 0,'STR' = 1,'INT' = 2,'DOUBLE' = 3,'BOOL' = 4,'MAP' = 5,'SLICE' = 6,'BYTES' = 7` ) -func encodeAttributes(attrs otelstorage.Attrs) string { +func encodeAttributes(attrs pcommon.Map) string { e := jx.GetEncoder() defer jx.PutEncoder(e) - encodeMap(e, attrs.AsMap()) + encodeMap(e, attrs) return e.String() } From 94f751606eb96beb9b4e5793dc9930eded8e2789 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Mon, 20 Nov 2023 20:56:03 +0300 Subject: [PATCH 011/112] feat(oteldb): properly setup Clickhouse storage --- cmd/oteldb/app.go | 13 +++---- cmd/oteldb/storage.go | 85 ++++++++++++++++++++++++------------------- 2 files changed, 53 insertions(+), 45 deletions(-) diff --git a/cmd/oteldb/app.go b/cmd/oteldb/app.go index 3def83b2..8d555491 100644 --- a/cmd/oteldb/app.go +++ b/cmd/oteldb/app.go @@ -31,7 +31,7 @@ import ( type App struct { services map[string]func(context.Context) error - storage + otelStorage lg *zap.Logger metrics Metrics @@ -50,18 +50,17 @@ func newApp(ctx context.Context, lg *zap.Logger, metrics Metrics) (_ *App, err e switch storageType { case "ch": - inserter, querier, err := setupCH(ctx, os.Getenv("CH_DSN"), lg, m) + store, err := setupCH(ctx, os.Getenv("CH_DSN"), lg, m) if err != nil { return nil, errors.Wrapf(err, "create storage %q", storageType) } - app.traceInserter = inserter - app.traceQuerier = querier + app.otelStorage = store case "yt", "": store, err := setupYT(ctx, lg, m) if err != nil { return nil, errors.Wrapf(err, "create storage %q", storageType) } - app.storage = store + app.otelStorage = store default: return nil, errors.Errorf("unknown storage %q", storageType) } @@ -191,7 +190,7 @@ func (app *App) trySetupLoki() error { } func (app *App) trySetupProm() error { - q := app.metricShard + q := app.metricsQuerier if q == nil { return nil } @@ -220,7 +219,7 @@ func (app *App) setupReceiver() error { if i := app.logInserter; i != nil { consumers.Logs = logstorage.NewConsumer(i) } - if c := app.metricConsumer; c != nil { + if c := app.metricsConsumer; c != nil { consumers.Metrics = c } diff --git a/cmd/oteldb/storage.go b/cmd/oteldb/storage.go index b7378df2..66dff4f4 100644 --- a/cmd/oteldb/storage.go +++ b/cmd/oteldb/storage.go @@ -13,6 +13,7 @@ import ( "github.com/cenkalti/backoff/v4" "github.com/go-faster/errors" "github.com/go-faster/sdk/zctx" + "github.com/prometheus/prometheus/storage" "go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp" "go.uber.org/zap" ytzap "go.ytsaurus.tech/library/go/core/log/zap" @@ -26,6 +27,7 @@ import ( "github.com/go-faster/oteldb/internal/logql/logqlengine" "github.com/go-faster/oteldb/internal/logstorage" "github.com/go-faster/oteldb/internal/metricsharding" + "github.com/go-faster/oteldb/internal/otelreceiver" "github.com/go-faster/oteldb/internal/otelstorage" "github.com/go-faster/oteldb/internal/tracestorage" "github.com/go-faster/oteldb/internal/yqlclient" @@ -45,18 +47,18 @@ func (q *combinedYTQuerier) SelectLogs(ctx context.Context, start, end otelstora return q.yql.SelectLogs(ctx, start, end, params) } -type storage struct { +type otelStorage struct { logQuerier logQuerier logInserter logstorage.Inserter traceQuerier traceQuerier traceInserter tracestorage.Inserter - metricShard *metricsharding.Sharder - metricConsumer *metricsharding.Consumer + metricsQuerier storage.Queryable + metricsConsumer otelreceiver.MetricsConsumer } -func setupYT(ctx context.Context, lg *zap.Logger, m Metrics) (storage, error) { +func setupYT(ctx context.Context, lg *zap.Logger, m Metrics) (otelStorage, error) { cfg := &yt.Config{ Logger: &ytzap.Logger{L: lg.Named("yc")}, DisableProxyDiscovery: true, @@ -68,16 +70,16 @@ func setupYT(ctx context.Context, lg *zap.Logger, m Metrics) (storage, error) { return setupYTQL(ctx, lg, m, cfg) } -func setupYQL(ctx context.Context, lg *zap.Logger, m Metrics, clusterName string, cfg *yt.Config) (storage, error) { +func setupYQL(ctx context.Context, lg *zap.Logger, m Metrics, clusterName string, cfg *yt.Config) (otelStorage, error) { zctx.From(ctx).Info("Setting up YQL") yc, err := ythttp.NewClient(cfg) if err != nil { - return storage{}, errors.Wrap(err, "yt") + return otelStorage{}, errors.Wrap(err, "yt") } proxy, err := cfg.GetProxy() if err != nil { - return storage{}, errors.Wrap(err, "get proxy addr") + return otelStorage{}, errors.Wrap(err, "get proxy addr") } yqlClient, err := yqlclient.NewClient("http://"+proxy, yqlclient.ClientOptions{ @@ -93,12 +95,12 @@ func setupYQL(ctx context.Context, lg *zap.Logger, m Metrics, clusterName string MeterProvider: m.MeterProvider(), }) if err != nil { - return storage{}, errors.Wrap(err, "create YQL client") + return otelStorage{}, errors.Wrap(err, "create YQL client") } tables := ytstorage.NewStaticTables(ypath.Path("//oteldb")) if err := migrateYT(ctx, yc, lg, tables); err != nil { - return storage{}, errors.Wrap(err, "migrate") + return otelStorage{}, errors.Wrap(err, "migrate") } inserter, err := ytstorage.NewInserter(yc, ytstorage.InserterOptions{ @@ -107,7 +109,7 @@ func setupYQL(ctx context.Context, lg *zap.Logger, m Metrics, clusterName string TracerProvider: m.TracerProvider(), }) if err != nil { - return storage{}, errors.Wrap(err, "create inserter") + return otelStorage{}, errors.Wrap(err, "create inserter") } engineQuerier, err := ytstorage.NewYQLQuerier(yqlClient, ytstorage.YQLQuerierOptions{ @@ -117,7 +119,7 @@ func setupYQL(ctx context.Context, lg *zap.Logger, m Metrics, clusterName string TracerProvider: m.TracerProvider(), }) if err != nil { - return storage{}, errors.Wrap(err, "create engine querier") + return otelStorage{}, errors.Wrap(err, "create engine querier") } labelQuerier, err := ytstorage.NewYTQLQuerier(yc, ytstorage.YTQLQuerierOptions{ @@ -126,7 +128,7 @@ func setupYQL(ctx context.Context, lg *zap.Logger, m Metrics, clusterName string TracerProvider: m.TracerProvider(), }) if err != nil { - return storage{}, errors.Wrap(err, "create label querier") + return otelStorage{}, errors.Wrap(err, "create label querier") } querier := &combinedYTQuerier{ @@ -137,26 +139,26 @@ func setupYQL(ctx context.Context, lg *zap.Logger, m Metrics, clusterName string shardOpts := metricsharding.ShardingOptions{} metricShard := metricsharding.NewSharder(yc, yqlClient, shardOpts) metricConsumer := metricsharding.NewConsumer(yc, shardOpts) - return storage{ - logQuerier: querier, - logInserter: inserter, - traceQuerier: querier, - traceInserter: inserter, - metricShard: metricShard, - metricConsumer: metricConsumer, + return otelStorage{ + logQuerier: querier, + logInserter: inserter, + traceQuerier: querier, + traceInserter: inserter, + metricsQuerier: metricShard, + metricsConsumer: metricConsumer, }, nil } -func setupYTQL(ctx context.Context, lg *zap.Logger, m Metrics, cfg *yt.Config) (storage, error) { +func setupYTQL(ctx context.Context, lg *zap.Logger, m Metrics, cfg *yt.Config) (otelStorage, error) { zctx.From(ctx).Info("Setting up YTQL") yc, err := ythttp.NewClient(cfg) if err != nil { - return storage{}, errors.Wrap(err, "yt") + return otelStorage{}, errors.Wrap(err, "yt") } tables := ytstorage.NewTables(ypath.Path("//oteldb")) if err := migrateYT(ctx, yc, lg, tables); err != nil { - return storage{}, errors.Wrap(err, "migrate") + return otelStorage{}, errors.Wrap(err, "migrate") } inserter, err := ytstorage.NewInserter(yc, ytstorage.InserterOptions{ @@ -165,7 +167,7 @@ func setupYTQL(ctx context.Context, lg *zap.Logger, m Metrics, cfg *yt.Config) ( TracerProvider: m.TracerProvider(), }) if err != nil { - return storage{}, errors.Wrap(err, "create inserter") + return otelStorage{}, errors.Wrap(err, "create inserter") } querier, err := ytstorage.NewYTQLQuerier(yc, ytstorage.YTQLQuerierOptions{ @@ -174,16 +176,16 @@ func setupYTQL(ctx context.Context, lg *zap.Logger, m Metrics, cfg *yt.Config) ( TracerProvider: m.TracerProvider(), }) if err != nil { - return storage{}, errors.Wrap(err, "create querier") + return otelStorage{}, errors.Wrap(err, "create querier") } - return storage{ - logQuerier: querier, - logInserter: inserter, - traceQuerier: querier, - traceInserter: inserter, - metricShard: nil, - metricConsumer: nil, + return otelStorage{ + logQuerier: querier, + logInserter: inserter, + traceQuerier: querier, + traceInserter: inserter, + metricsQuerier: nil, + metricsConsumer: nil, }, nil } @@ -211,10 +213,10 @@ func setupCH( dsn string, lg *zap.Logger, m Metrics, -) (*chstorage.Inserter, *chstorage.Querier, error) { +) (store otelStorage, _ error) { u, err := url.Parse(dsn) if err != nil { - return nil, nil, errors.Wrap(err, "parse DSN") + return store, errors.Wrap(err, "parse DSN") } pass, _ := u.User.Password() @@ -246,7 +248,7 @@ func setupCH( return c, nil }, connectBackoff) if err != nil { - return nil, nil, errors.Wrap(err, "migrate") + return store, errors.Wrap(err, "migrate") } tables := chstorage.Tables{ @@ -254,7 +256,7 @@ func setupCH( Tags: "traces_tags", } if err := tables.Create(ctx, c); err != nil { - return nil, nil, errors.Wrap(err, "create tables") + return store, errors.Wrap(err, "create tables") } inserter, err := chstorage.NewInserter(c, chstorage.InserterOptions{ @@ -263,7 +265,7 @@ func setupCH( TracerProvider: m.TracerProvider(), }) if err != nil { - return nil, nil, errors.Wrap(err, "create inserter") + return store, errors.Wrap(err, "create inserter") } querier, err := chstorage.NewQuerier(c, chstorage.QuerierOptions{ @@ -272,8 +274,15 @@ func setupCH( TracerProvider: m.TracerProvider(), }) if err != nil { - return nil, nil, errors.Wrap(err, "create querier") + return store, errors.Wrap(err, "create querier") } - return inserter, querier, nil + return otelStorage{ + logQuerier: nil, + logInserter: nil, + traceQuerier: querier, + traceInserter: inserter, + metricsQuerier: querier, + metricsConsumer: inserter, + }, nil } From 229559dffc453f097fa584dee7dca5a64efb37db Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 20 Nov 2023 23:00:06 +0300 Subject: [PATCH 012/112] fix(chstorage.querier): use tracer provider --- internal/chstorage/querier.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/chstorage/querier.go b/internal/chstorage/querier.go index a5aff8e6..b44d19a0 100644 --- a/internal/chstorage/querier.go +++ b/internal/chstorage/querier.go @@ -47,6 +47,6 @@ func NewQuerier(c *chpool.Pool, opts QuerierOptions) (*Querier, error) { return &Querier{ ch: c, tables: opts.Tables, - tracer: otel.Tracer("chstorage.Querier"), + tracer: opts.TracerProvider.Tracer("chstorage.Querier"), }, nil } From f2317eac9221fd2c8d5018fbc2cf775bd29ae0a1 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Mon, 20 Nov 2023 23:01:39 +0300 Subject: [PATCH 013/112] feat(chstorage): insert labels too --- internal/chstorage/columns_metrics.go | 26 ++++++++++++ internal/chstorage/inserter_metrics.go | 58 ++++++++++++++++++++++---- 2 files changed, 75 insertions(+), 9 deletions(-) diff --git a/internal/chstorage/columns_metrics.go b/internal/chstorage/columns_metrics.go index bdb15a41..90973cd0 100644 --- a/internal/chstorage/columns_metrics.go +++ b/internal/chstorage/columns_metrics.go @@ -47,3 +47,29 @@ func (c *metricColumns) Result() proto.Results { {Name: "resource", Data: &c.resource}, } } + +type labelsColumns struct { + name *proto.ColLowCardinality[string] + value proto.ColStr +} + +func newLabelsColumns() *labelsColumns { + return &labelsColumns{ + name: new(proto.ColStr).LowCardinality(), + } +} + +func (c *labelsColumns) Input() proto.Input { + input := proto.Input{ + {Name: "name", Data: c.name}, + {Name: "value", Data: c.value}, + } + return input +} + +func (c *labelsColumns) Result() proto.Results { + return proto.Results{ + {Name: "name", Data: c.name}, + {Name: "value", Data: &c.value}, + } +} diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index 605df658..9f744111 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -7,27 +7,65 @@ import ( "github.com/go-faster/errors" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" + "golang.org/x/sync/errgroup" ) // ConsumeMetrics inserts given metrics. func (i *Inserter) ConsumeMetrics(ctx context.Context, metrics pmetric.Metrics) error { - c := newMetricColumns() - if err := i.mapMetrics(c, metrics); err != nil { + var ( + points = newMetricColumns() + + labels = newLabelsColumns() + collectLabels = func(m pcommon.Map) { + m.Range(func(k string, v pcommon.Value) bool { + labels.name.Append(k) + // FIXME(tdakkota): annoying allocations + labels.value.Append(v.AsString()) + return true + }) + } + ) + + if err := i.mapMetrics(points, metrics, collectLabels); err != nil { return errors.Wrap(err, "map metrics") } - input := c.Input() - if err := i.ch.Do(ctx, ch.Query{ - Body: input.Into(i.tables.Points), - Input: input, - }); err != nil { - return errors.Wrap(err, "insert") + { + grp, grpCtx := errgroup.WithContext(ctx) + + grp.Go(func() error { + ctx := grpCtx + + input := points.Input() + if err := i.ch.Do(ctx, ch.Query{ + Body: input.Into(i.tables.Points), + Input: input, + }); err != nil { + return errors.Wrap(err, "insert points") + } + return nil + }) + grp.Go(func() error { + ctx := grpCtx + + input := labels.Input() + if err := i.ch.Do(ctx, ch.Query{ + Body: input.Into(i.tables.Labels), + Input: input, + }); err != nil { + return errors.Wrap(err, "insert labels") + } + return nil + }) + if err := grp.Wait(); err != nil { + return err + } } return nil } -func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics) error { +func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics, collectLabels func(attrs pcommon.Map)) error { var ( addPoints = func( name string, @@ -50,6 +88,7 @@ func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics) error { return errors.Errorf("unexpected metric %q value type: %v", name, typ) } + collectLabels(attrs) c.name.Append(name) c.ts.Append(ts) c.value.Append(val) @@ -64,6 +103,7 @@ func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics) error { for i := 0; i < resMetrics.Len(); i++ { resMetric := resMetrics.At(i) resAttrs := resMetric.Resource().Attributes() + collectLabels(resAttrs) scopeMetrics := resMetric.ScopeMetrics() for i := 0; i < scopeMetrics.Len(); i++ { From f24cd3ee2e1500ddf54381707a86d42fa8b6e91f Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 20 Nov 2023 23:09:51 +0300 Subject: [PATCH 014/112] chore: fix lint issues --- cmd/chotel/main.go | 3 +++ integration/chotele2e/e2e_test.go | 4 ++-- internal/chtrace/chtrace.go | 5 +++++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/cmd/chotel/main.go b/cmd/chotel/main.go index 02c58bc5..55771e5e 100644 --- a/cmd/chotel/main.go +++ b/cmd/chotel/main.go @@ -40,6 +40,7 @@ func main() { }) } +// App is the trace exporter application. type App struct { log *zap.Logger metrics *app.Metrics @@ -64,6 +65,7 @@ const DDL = `CREATE TABLE IF NOT EXISTS opentelemetry_span_export TTL toStartOfMinute(exported_at) + INTERVAL 10 MINUTE ` +// NewApp initializes the trace exporter application. func NewApp(logger *zap.Logger, metrics *app.Metrics) (*App, error) { a := &App{ log: logger, @@ -81,6 +83,7 @@ func NewApp(logger *zap.Logger, metrics *app.Metrics) (*App, error) { return a, nil } +// Run starts and runs the application. func (a *App) Run(ctx context.Context) error { ctx = zctx.Base(ctx, a.log) if err := a.setup(ctx); err != nil { diff --git a/integration/chotele2e/e2e_test.go b/integration/chotele2e/e2e_test.go index 8bd3320e..2de196cb 100644 --- a/integration/chotele2e/e2e_test.go +++ b/integration/chotele2e/e2e_test.go @@ -30,7 +30,7 @@ type randomIDGenerator struct { } // NewSpanID returns a non-zero span ID from a randomly-chosen sequence. -func (gen *randomIDGenerator) NewSpanID(ctx context.Context, traceID trace.TraceID) (sid trace.SpanID) { +func (gen *randomIDGenerator) NewSpanID(context.Context, trace.TraceID) (sid trace.SpanID) { gen.Lock() defer gen.Unlock() gen.rand.Read(sid[:]) @@ -39,7 +39,7 @@ func (gen *randomIDGenerator) NewSpanID(ctx context.Context, traceID trace.Trace // NewIDs returns a non-zero trace ID and a non-zero span ID from a // randomly-chosen sequence. -func (gen *randomIDGenerator) NewIDs(ctx context.Context) (tid trace.TraceID, sid trace.SpanID) { +func (gen *randomIDGenerator) NewIDs(context.Context) (tid trace.TraceID, sid trace.SpanID) { gen.Lock() defer gen.Unlock() gen.rand.Read(tid[:]) diff --git a/internal/chtrace/chtrace.go b/internal/chtrace/chtrace.go index 925a15d2..d9589379 100644 --- a/internal/chtrace/chtrace.go +++ b/internal/chtrace/chtrace.go @@ -21,6 +21,7 @@ type Table struct { Attributes proto.ColMap[string, string] // attribute } +// Rows returns Trace per row. func (t Table) Rows() []Trace { var out []Trace for i := 0; i < t.TraceID.Rows(); i++ { @@ -40,6 +41,7 @@ func (t Table) Rows() []Trace { return out } +// Trace is a single trace span. type Trace struct { TraceID trace.TraceID SpanID trace.SpanID @@ -50,6 +52,7 @@ type Trace struct { Attributes map[string]string } +// Result returns proto.Results for Table. func (t *Table) Result() proto.Results { return proto.Results{ {Name: "trace_id", Data: &t.TraceID}, @@ -63,6 +66,7 @@ func (t *Table) Result() proto.Results { } } +// Columns returns column names (and mappings) for doing SELECTS. func (t *Table) Columns() []string { var out []string for _, v := range t.Result() { @@ -78,6 +82,7 @@ func (t *Table) Columns() []string { return out } +// NewTable creates and initializes new Table. func NewTable() *Table { return &Table{ OperationName: new(proto.ColStr).LowCardinality(), From cc133718be35cb039daf6f6224a31c60dcfed460 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 21 Nov 2023 12:53:43 +0300 Subject: [PATCH 015/112] feat(chstorage): label querier --- internal/chstorage/querier_metrics.go | 107 ++++++++++++++++++++++++-- 1 file changed, 101 insertions(+), 6 deletions(-) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 075d9552..d290910d 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -56,17 +56,112 @@ var _ storage.Querier = (*promQuerier)(nil) // It is not safe to use the strings beyond the lifetime of the querier. // If matchers are specified the returned result set is reduced // to label values of metrics matching the matchers. -func (p *promQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - // TODO: Implement - return nil, nil, nil +func (p *promQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) (result []string, _ annotations.Annotations, rerr error) { + table := p.tables.Labels + + ctx, span := p.tracer.Start(ctx, "LabelValues", + trace.WithAttributes( + attribute.String("chstorage.table", table), + attribute.String("chstorage.label_to_query", name), + attribute.Int("chstorage.label_matchers", len(matchers)), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + var query strings.Builder + fmt.Fprintf(&query, "SELECT DISTINCT value FROM %#q WHERE name = %s\n", table, singleQuoted(name)) + if err := addLabelMatchers(&query, matchers); err != nil { + return nil, nil, err + } + + var column proto.ColStr + if err := p.ch.Do(ctx, ch.Query{ + Body: query.String(), + Result: proto.Results{ + {Name: "value", Data: &column}, + }, + OnResult: func(ctx context.Context, block proto.Block) error { + for i := 0; i < column.Rows(); i++ { + result = append(result, column.Row(i)) + } + return nil + }, + }); err != nil { + return nil, nil, errors.Wrap(err, "do query") + } + return result, nil, nil } // LabelNames returns all the unique label names present in the block in sorted order. // If matchers are specified the returned result set is reduced // to label names of metrics matching the matchers. -func (p *promQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - // TODO: Implement - return nil, nil, nil +func (p *promQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) (result []string, _ annotations.Annotations, rerr error) { + table := p.tables.Labels + + ctx, span := p.tracer.Start(ctx, "LabelNames", + trace.WithAttributes( + attribute.String("chstorage.table", table), + attribute.Int("chstorage.label_matchers", len(matchers)), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + var query strings.Builder + fmt.Fprintf(&query, "SELECT DISTINCT name FROM %#q WHERE true\n", table) + if err := addLabelMatchers(&query, matchers); err != nil { + return nil, nil, err + } + + var column proto.ColStr + if err := p.ch.Do(ctx, ch.Query{ + Body: query.String(), + Result: proto.Results{ + {Name: "name", Data: &column}, + }, + OnResult: func(ctx context.Context, block proto.Block) error { + for i := 0; i < column.Rows(); i++ { + result = append(result, column.Row(i)) + } + return nil + }, + }); err != nil { + return nil, nil, errors.Wrap(err, "do query") + } + return result, nil, nil +} + +func addLabelMatchers(query *strings.Builder, matchers []*labels.Matcher) error { + for _, m := range matchers { + switch m.Type { + case labels.MatchEqual, labels.MatchRegexp: + query.WriteString("AND ") + case labels.MatchNotEqual, labels.MatchNotRegexp: + query.WriteString("AND NOT ") + default: + return errors.Errorf("unexpected type %q", m.Type) + } + + // Note: predicate negated above. + switch m.Type { + case labels.MatchEqual, labels.MatchNotEqual: + fmt.Fprintf(query, "name = %s\n", singleQuoted(m.Value)) + case labels.MatchRegexp, labels.MatchNotRegexp: + fmt.Fprintf(query, "name REGEXP %s\n", singleQuoted(m.Value)) + default: + return errors.Errorf("unexpected type %q", m.Type) + } + } + return nil } // Close releases the resources of the Querier. From c446b013acb7e489fdfc42f6535e1614c40e277d Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 21 Nov 2023 14:50:18 +0300 Subject: [PATCH 016/112] fix(tempoe2e): set Clickhouse table names --- integration/tempoe2e/ch_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/integration/tempoe2e/ch_test.go b/integration/tempoe2e/ch_test.go index efcdcf38..9e718ef8 100644 --- a/integration/tempoe2e/ch_test.go +++ b/integration/tempoe2e/ch_test.go @@ -66,6 +66,9 @@ func TestCH(t *testing.T) { tables := chstorage.Tables{ Spans: prefix + "_spans", Tags: prefix + "_tags", + + Points: prefix + "_points", + Labels: prefix + "_labels", } t.Logf("Test tables prefix: %s", prefix) require.NoError(t, tables.Create(ctx, c)) From fe3ed21e5c97f96628f008f78e64fb1ef416d669 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 21 Nov 2023 14:55:24 +0300 Subject: [PATCH 017/112] fix(chstorage): validate table name before creation --- internal/chstorage/schema.go | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/internal/chstorage/schema.go b/internal/chstorage/schema.go index 4f9a0fff..cb0f7b1d 100644 --- a/internal/chstorage/schema.go +++ b/internal/chstorage/schema.go @@ -17,6 +17,32 @@ type Tables struct { Labels string } +// Validate checks table names +func (t Tables) Validate() error { + validateTableName := func(name string) error { + if name == "" { + return errors.New("table name must be non-empty") + } + return nil + } + + for _, table := range []struct { + name string + fieldName string + }{ + {t.Spans, "Spans"}, + {t.Tags, "Tags"}, + + {t.Points, "Points"}, + {t.Labels, "Labels"}, + } { + if err := validateTableName(table.name); err != nil { + return errors.Wrapf(err, "table %s", table.fieldName) + } + } + return nil +} + var defaultTables = Tables{ Spans: "traces_spans", Tags: "traces_tags", @@ -31,6 +57,10 @@ type chClient interface { // Create creates tables. func (t Tables) Create(ctx context.Context, c chClient) error { + if err := t.Validate(); err != nil { + return errors.Wrap(err, "validate") + } + type schema struct { name string query string From c78289e566551e14b7b76be990c11eb1a2fdd9fc Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 21 Nov 2023 14:58:29 +0300 Subject: [PATCH 018/112] fix(chstorage): use right column name --- internal/chstorage/columns_metrics.go | 6 +++--- internal/chstorage/schema_metrics.go | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/internal/chstorage/columns_metrics.go b/internal/chstorage/columns_metrics.go index 90973cd0..65dee3ca 100644 --- a/internal/chstorage/columns_metrics.go +++ b/internal/chstorage/columns_metrics.go @@ -20,7 +20,7 @@ func newMetricColumns() *metricColumns { func (c *metricColumns) StaticColumns() []string { return []string{ "name", - "ts", + "timestamp", "value", "attributes", "resource", @@ -30,7 +30,7 @@ func (c *metricColumns) StaticColumns() []string { func (c *metricColumns) Input() proto.Input { input := proto.Input{ {Name: "name", Data: c.name}, - {Name: "ts", Data: c.ts}, + {Name: "timestamp", Data: c.ts}, {Name: "value", Data: c.value}, {Name: "attributes", Data: c.attributes}, {Name: "resource", Data: c.resource}, @@ -41,7 +41,7 @@ func (c *metricColumns) Input() proto.Input { func (c *metricColumns) Result() proto.Results { return proto.Results{ {Name: "name", Data: c.name}, - {Name: "ts", Data: c.ts}, + {Name: "timestamp", Data: c.ts}, {Name: "value", Data: &c.value}, {Name: "attributes", Data: &c.attributes}, {Name: "resource", Data: &c.resource}, diff --git a/internal/chstorage/schema_metrics.go b/internal/chstorage/schema_metrics.go index 1ab4aa0f..a4c6b0cf 100644 --- a/internal/chstorage/schema_metrics.go +++ b/internal/chstorage/schema_metrics.go @@ -17,7 +17,7 @@ const ( resource String ) ENGINE = MergeTree() - ORDER BY ts;` + ORDER BY timestamp;` labelsSchema = `CREATE TABLE IF NOT EXISTS %s ( name LowCardinality(String), From a487d36d423dd5369dd89a43c65c329bca544f0b Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 21 Nov 2023 16:25:38 +0300 Subject: [PATCH 019/112] ci(lint): fix issues --- .golangci.yml | 4 ++-- internal/chstorage/querier_metrics.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index b2f70ed5..fb656ff7 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -135,8 +135,8 @@ issues: text: "dupSubExpr" # False positive. - - path: internal/metricsharding/prometheus\.go - linters: [govet] + - linters: [govet] + source: func.+Seek\(.+int64\).+chunkenc\.ValueType text: "method Seek" # Not gonna use const, does not make any sense. diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index d290910d..0da110fb 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -171,7 +171,7 @@ func (p *promQuerier) Close() error { // Select returns a set of series that matches the given label matchers. // Caller can specify if it requires returned series to be sorted. Prefer not requiring sorting for better performance. -// It allows passing hints that can help in optimising select, but it's up to implementation how this is used if used at all. +// It allows passing hints that can help in optimizing select, but it's up to implementation how this is used if used at all. func (p *promQuerier) Select(ctx context.Context, _ bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { ss, err := p.selectSeries(ctx, hints, matchers...) if err != nil { From 03def16f86a019e6f893fa256df6d425bce9db18 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 00:02:03 +0300 Subject: [PATCH 020/112] fix(chstorage): handle empty value type --- internal/chstorage/inserter_metrics.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index 9f744111..e8d6303b 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -79,6 +79,9 @@ func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics, collect var val float64 switch typ := point.ValueType(); typ { + case pmetric.NumberDataPointValueTypeEmpty: + // Just ignore it. + continue case pmetric.NumberDataPointValueTypeInt: // TODO(tdakkota): check for overflow val = float64(point.IntValue()) From 1248c1962906838c607f2b01ac95857cdb583783 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 00:02:21 +0300 Subject: [PATCH 021/112] fix(chstorage): use right column type --- internal/chstorage/querier_metrics.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 0da110fb..060eae0b 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -122,11 +122,11 @@ func (p *promQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matche return nil, nil, err } - var column proto.ColStr + column := new(proto.ColStr).LowCardinality() if err := p.ch.Do(ctx, ch.Query{ Body: query.String(), Result: proto.Results{ - {Name: "name", Data: &column}, + {Name: "name", Data: column}, }, OnResult: func(ctx context.Context, block proto.Block) error { for i := 0; i < column.Rows(); i++ { From b9935580210c3e161cad12d30769017b49a67007 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 21 Nov 2023 22:17:11 +0300 Subject: [PATCH 022/112] test(prome2e): add basic Prometheus API test --- integration/prome2e/ch_test.go | 83 +++++++++++++++++++++ integration/prome2e/common_test.go | 102 +++++++++++++++++++++++++ integration/prome2e/prom_e2e.go | 115 +++++++++++++++++++++++++++++ 3 files changed, 300 insertions(+) create mode 100644 integration/prome2e/ch_test.go create mode 100644 integration/prome2e/common_test.go create mode 100644 integration/prome2e/prom_e2e.go diff --git a/integration/prome2e/ch_test.go b/integration/prome2e/ch_test.go new file mode 100644 index 00000000..341087d5 --- /dev/null +++ b/integration/prome2e/ch_test.go @@ -0,0 +1,83 @@ +package prome2e_test + +import ( + "context" + "os" + "strings" + "testing" + "time" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/chpool" + "github.com/cenkalti/backoff/v4" + "github.com/go-faster/errors" + "github.com/google/uuid" + "github.com/stretchr/testify/require" + "github.com/testcontainers/testcontainers-go" + + "github.com/go-faster/oteldb/internal/chstorage" +) + +func TestCH(t *testing.T) { + t.Parallel() + if os.Getenv("E2E") == "" { + t.Skip("Set E2E env to run") + } + ctx := context.Background() + + req := testcontainers.ContainerRequest{ + Name: "oteldb-prome2e-clickhouse", + Image: "clickhouse/clickhouse-server:23.10", + ExposedPorts: []string{"8123/tcp", "9000/tcp"}, + } + chContainer, err := testcontainers.GenericContainer(ctx, testcontainers.GenericContainerRequest{ + ContainerRequest: req, + Started: true, + Logger: testcontainers.TestLogger(t), + Reuse: true, + }) + require.NoError(t, err, "container start") + + endpoint, err := chContainer.PortEndpoint(ctx, "9000", "") + require.NoError(t, err, "container endpoint") + + opts := ch.Options{ + Address: endpoint, + Database: "default", + } + + connectBackoff := backoff.NewExponentialBackOff() + connectBackoff.InitialInterval = 2 * time.Second + connectBackoff.MaxElapsedTime = time.Minute + c, err := backoff.RetryWithData(func() (*chpool.Pool, error) { + c, err := chpool.Dial(ctx, chpool.Options{ + ClientOptions: opts, + }) + if err != nil { + return nil, errors.Wrap(err, "dial") + } + return c, nil + }, connectBackoff) + if err != nil { + t.Fatal(err) + } + + prefix := "traces_" + strings.ReplaceAll(uuid.NewString(), "-", "") + tables := chstorage.Tables{ + Spans: prefix + "_spans", + Tags: prefix + "_tags", + + Points: prefix + "_points", + Labels: prefix + "_labels", + } + t.Logf("Test tables prefix: %s", prefix) + require.NoError(t, tables.Create(ctx, c)) + + inserter, err := chstorage.NewInserter(c, chstorage.InserterOptions{Tables: tables}) + require.NoError(t, err) + + querier, err := chstorage.NewQuerier(c, chstorage.QuerierOptions{Tables: tables}) + require.NoError(t, err) + + runTest(ctx, t, inserter, querier) +} diff --git a/integration/prome2e/common_test.go b/integration/prome2e/common_test.go new file mode 100644 index 00000000..564e8778 --- /dev/null +++ b/integration/prome2e/common_test.go @@ -0,0 +1,102 @@ +package prome2e_test + +import ( + "context" + "net/http/httptest" + "os" + "testing" + + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/storage" + "github.com/stretchr/testify/require" + + "github.com/go-faster/oteldb/integration/prome2e" + "github.com/go-faster/oteldb/internal/otelreceiver" + "github.com/go-faster/oteldb/internal/promapi" + "github.com/go-faster/oteldb/internal/promhandler" +) + +func readBatchSet(p string) (s prome2e.BatchSet, _ error) { + f, err := os.Open(p) + if err != nil { + return s, err + } + defer func() { + _ = f.Close() + }() + return prome2e.ParseBatchSet(f) +} + +func setupDB( + ctx context.Context, + t *testing.T, + set prome2e.BatchSet, + consumer otelreceiver.MetricsConsumer, + querier storage.Queryable, +) *promapi.Client { + for i, b := range set.Batches { + if err := consumer.ConsumeMetrics(ctx, b); err != nil { + t.Fatalf("Send batch %d: %+v", i, err) + } + } + + engine := promql.NewEngine(promql.EngineOpts{}) + api := promhandler.NewPromAPI(engine, querier, promhandler.PromAPIOptions{}) + promh, err := promapi.NewServer(api) + require.NoError(t, err) + + s := httptest.NewServer(promh) + t.Cleanup(s.Close) + + c, err := promapi.NewClient(s.URL, promapi.WithClient(s.Client())) + require.NoError(t, err) + return c +} + +func runTest( + ctx context.Context, + t *testing.T, + consumer otelreceiver.MetricsConsumer, + querier storage.Queryable, +) { + set, err := readBatchSet("_testdata/metrics.json") + require.NoError(t, err) + require.NotEmpty(t, set.Batches) + require.NotEmpty(t, set.Labels) + c := setupDB(ctx, t, set, consumer, querier) + + t.Run("Labels", func(t *testing.T) { + t.Run("GetLabels", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetLabels(ctx, promapi.GetLabelsParams{}) + a.NoError(err) + a.Len(r.Data, len(set.Labels)) + for _, label := range r.Data { + a.Contains(set.Labels, label) + } + }) + t.Run("PostLabels", func(t *testing.T) { + a := require.New(t) + + r, err := c.PostLabels(ctx, &promapi.LabelsForm{}) + a.NoError(err) + a.Len(r.Data, len(set.Labels)) + for _, label := range r.Data { + a.Contains(set.Labels, label) + } + }) + }) + t.Run("LabelValues", func(t *testing.T) { + a := require.New(t) + + for labelName, valueSet := range set.Labels { + r, err := c.GetLabelValues(ctx, promapi.GetLabelValuesParams{Label: labelName}) + a.NoError(err) + a.Len(r.Data, len(valueSet)) + for _, val := range r.Data { + a.Containsf(valueSet, val, "check label %q", labelName) + } + } + }) +} diff --git a/integration/prome2e/prom_e2e.go b/integration/prome2e/prom_e2e.go new file mode 100644 index 00000000..b7134422 --- /dev/null +++ b/integration/prome2e/prom_e2e.go @@ -0,0 +1,115 @@ +// Package prome2e provides scripts for E2E testing Prometheus API implementation. +package prome2e + +import ( + "io" + + "github.com/go-faster/errors" + "github.com/go-faster/jx" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +// BatchSet is a set of batches. +type BatchSet struct { + Batches []pmetric.Metrics + + Labels map[string]map[string]struct{} +} + +// ParseBatchSet parses JSON batches from given reader. +func ParseBatchSet(r io.Reader) (s BatchSet, _ error) { + d := jx.Decode(r, 4096) + u := pmetric.JSONUnmarshaler{} + + for d.Next() != jx.Invalid { + data, err := d.Raw() + if err != nil { + return s, errors.Wrap(err, "read line") + } + + raw, err := u.UnmarshalMetrics(data) + if err != nil { + return s, errors.Wrap(err, "parse batch") + } + + if err := s.addBatch(raw); err != nil { + return s, errors.Wrap(err, "add batch") + } + } + return s, nil +} + +func (s *BatchSet) addBatch(raw pmetric.Metrics) error { + s.Batches = append(s.Batches, raw) + + resMetrics := raw.ResourceMetrics() + for i := 0; i < resMetrics.Len(); i++ { + resLog := resMetrics.At(i) + res := resLog.Resource() + s.addLabels(res.Attributes()) + + scopeMetrics := resLog.ScopeMetrics() + for i := 0; i < scopeMetrics.Len(); i++ { + scopeMetrics := scopeMetrics.At(i) + scope := scopeMetrics.Scope() + s.addLabels(scope.Attributes()) + + metrics := scopeMetrics.Metrics() + for i := 0; i < metrics.Len(); i++ { + metric := metrics.At(i) + if err := s.addMetric(metric); err != nil { + return errors.Wrap(err, "add metric") + } + } + } + } + return nil +} + +func (s *BatchSet) addMetric(metric pmetric.Metric) error { + switch t := metric.Type(); t { + case pmetric.MetricTypeGauge: + points := metric.Gauge().DataPoints() + for i := 0; i < points.Len(); i++ { + point := points.At(i) + s.addLabels(point.Attributes()) + } + case pmetric.MetricTypeSum: + points := metric.Sum().DataPoints() + for i := 0; i < points.Len(); i++ { + point := points.At(i) + s.addLabels(point.Attributes()) + } + case pmetric.MetricTypeHistogram: + case pmetric.MetricTypeExponentialHistogram: + case pmetric.MetricTypeSummary: + case pmetric.MetricTypeEmpty: + default: + return errors.Errorf("unexpected type %v", t) + } + return nil +} + +func (s *BatchSet) addLabels(m pcommon.Map) { + m.Range(func(k string, v pcommon.Value) bool { + switch t := v.Type(); t { + case pcommon.ValueTypeMap, pcommon.ValueTypeSlice: + default: + s.addLabel(k, v.AsString()) + } + return true + }) +} + +func (s *BatchSet) addLabel(label, val string) { + if s.Labels == nil { + s.Labels = map[string]map[string]struct{}{} + } + m := s.Labels[label] + if m == nil { + m = map[string]struct{}{} + s.Labels[label] = m + } + m[val] = struct{}{} +} From b7db6765e42de66e7f3414b08e30dac3ca097f6b Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 00:24:56 +0300 Subject: [PATCH 023/112] chore(prome2e): add metrics testdata --- integration/prome2e/_testdata/metrics.json | 173 +++++++++++++++++++++ 1 file changed, 173 insertions(+) create mode 100755 integration/prome2e/_testdata/metrics.json diff --git a/integration/prome2e/_testdata/metrics.json b/integration/prome2e/_testdata/metrics.json new file mode 100755 index 00000000..0a6d186e --- /dev/null +++ b/integration/prome2e/_testdata/metrics.json @@ -0,0 +1,173 @@ +{"resourceMetrics":[{"resource":{"attributes":[{"key":"process.runtime.description","value":{"stringValue":"go version go1.21.3 linux/amd64"}},{"key":"process.runtime.name","value":{"stringValue":"go"}},{"key":"process.runtime.version","value":{"stringValue":"go1.21.3"}},{"key":"service.name","value":{"stringValue":"go-faster.oteldb.chotel"}},{"key":"telemetry.sdk.language","value":{"stringValue":"go"}},{"key":"telemetry.sdk.name","value":{"stringValue":"opentelemetry"}},{"key":"telemetry.sdk.version","value":{"stringValue":"1.21.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/runtime","version":"0.46.0"},"metrics":[{"name":"runtime.uptime","description":"Milliseconds since application was initialized","unit":"ms","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668813332","timeUnixNano":"1700601637670044355","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.goroutines","description":"Number of goroutines that currently exist","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668819882","timeUnixNano":"1700601637670045715","asInt":"17"}],"aggregationTemporality":2}},{"name":"process.runtime.go.cgo.calls","description":"Number of cgo calls made by the current process","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668822072","timeUnixNano":"1700601637670046075","asInt":"0"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_alloc","description":"Bytes of allocated heap objects","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668829062","timeUnixNano":"1700601637670046275","asInt":"2058040"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_idle","description":"Bytes in idle (unused) spans","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668830742","timeUnixNano":"1700601637670046445","asInt":"3981312"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_inuse","description":"Bytes in in-use spans","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668832422","timeUnixNano":"1700601637670046605","asInt":"3620864"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_objects","description":"Number of allocated heap objects","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668833752","timeUnixNano":"1700601637670046765","asInt":"6723"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_released","description":"Bytes of idle spans whose physical memory has been returned to the OS","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668835252","timeUnixNano":"1700601637670046935","asInt":"3883008"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_sys","description":"Bytes of heap memory obtained from the OS","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668837592","timeUnixNano":"1700601637670047085","asInt":"7602176"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.lookups","description":"Number of pointer lookups performed by the runtime","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668859732","timeUnixNano":"1700601637670047265","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.mem.live_objects","description":"Number of live objects is the number of cumulative Mallocs - Frees","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668863242","timeUnixNano":"1700601637670047445","asInt":"6723"}],"aggregationTemporality":2}},{"name":"process.runtime.go.gc.count","description":"Number of completed garbage collection cycles","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668864812","timeUnixNano":"1700601637670052705","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.gc.pause_total_ns","description":"Cumulative nanoseconds in GC stop-the-world pauses since the program started","sum":{"dataPoints":[{"startTimeUnixNano":"1700601637668875752","timeUnixNano":"1700601637670052945","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.21.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601643109538409","asInt":"3"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601643109540069","count":"3","sum":38,"bucketCounts":["1","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":8,"max":17}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601643109543989","asInt":"3"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601643109545999","asInt":"25"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601643109550029","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601643109550719","asInt":"13"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601643109551409","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601643109553549","asDouble":4.999427004}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601643109360351","asInt":"47731872"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601643109555649","asInt":"60301184"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601643109375201","asInt":"83353880"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601643109556389","asDouble":0.13}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601643109517929","asInt":"146800640"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601643109559799","count":"4","sum":20176032,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1601557,"max":9110550},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601643109559799","count":"1","sum":620981,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","1"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":620981,"max":620981}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601643109561999","count":"4","sum":8,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601643109563169","count":"4","sum":32408,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","2","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":4796,"max":11408}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601643109564259","count":"1","sum":0,"bucketCounts":["1","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601643109564259","count":"4","sum":4,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601643109565299","count":"4","sum":4,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601643109565299","count":"1","sum":0,"bucketCounts":["1","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601643109566209","count":"2","sum":635181,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":236267,"max":398914},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601643109566209","count":"1","sum":2029081,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","1"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2029081,"max":2029081}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601643109567468","count":"1","sum":1931,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":1931},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601643109567468","count":"2","sum":16190,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","1","0","0","1"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":4794,"max":11396}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601643109568838","count":"1","sum":2,"bucketCounts":["0","1","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601643109568838","count":"2","sum":4,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601643109569868","count":"1","sum":1,"bucketCounts":["0","1","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601643109569868","count":"2","sum":2,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601643109571028","count":"1","sum":1,"bucketCounts":["0","1","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601643109571028","count":"2","sum":2,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601643109523109","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601643109534559","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601643109530279","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601643109530969","asInt":"10000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601643109525599","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601643109535209","asInt":"1000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601643109577448","asInt":"25"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601643109577448","asInt":"25"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601643109578068","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601643109578068","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601643109579568","asInt":"13"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601643109579568","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601643109579568","asInt":"13"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601643109580258","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601643109580258","asInt":"13"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601643109580258","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601644465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601644465000000","asDouble":0.000645861}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601644465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601644465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601644465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601645439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601645439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601645439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601645439000000","asDouble":0.000896027}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601645439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":395}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":56618}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601646616000000","asDouble":1}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601645439}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000"}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":4923392}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":136345}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601646616000000","asDouble":1}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601637.3167245}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":15107416}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":23396352}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","count":"5","sum":0.000945096,"quantileValues":[{"value":0.000075919},{"quantile":0.25,"value":0.000154358},{"quantile":0.5,"value":0.000213016},{"quantile":0.75,"value":0.000245897},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601645}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1048576}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":404}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1460116}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0.09}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","count":"4","sum":0.003234394,"bucketCounts":["4","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","count":"4","sum":0,"bucketCounts":["4","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":79727}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":404}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":395}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":4}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":46800}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":15107416}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":822448128}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601645}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":14}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":404}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601636.42}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":4}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":4}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":14352}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":4866048}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":408000}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":3433348}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":20}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":24302384}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0.006182092}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":66584576}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":8673632}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":39187720}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000"}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1769472}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601646616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":404},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":33}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":38400}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1769472}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1}]}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":395}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":404}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":404}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":8}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1700601637670}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":18472960}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":395680}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":10000}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":22080232}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":23}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601646616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646616000000","asDouble":1}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":13.763416162999999}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"2","sum":2,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"4","sum":4,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601646873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601646873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601646873000000","asDouble":10000}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":151257088}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":151},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":151}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"1","sum":652,"bucketCounts":["0","0","0","0","0","0","0","0","0","1","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"8","sum":66166,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","4","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":6}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":326},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":326}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"2","sum":2296338,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"4","sum":1302221,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"2","sum":4,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"4","sum":8,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":0.004400018}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":60},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":266}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"8","sum":39809501,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"5","sum":3544530,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"2","sum":15247,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","1"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"4","sum":32859,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","2","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601646873000000","asDouble":1}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":1}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":398}]}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"8","sum":16,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"8","sum":8,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"5","sum":1,"bucketCounts":["4","1","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":83616024}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":9}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"2","sum":2,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"4","sum":4,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0.3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601646873000000","asDouble":57624384}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":130494880}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"9","sum":477,"bucketCounts":["3","3","1","0","0","1","1","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":151}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"8","sum":8,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","count":"5","sum":0,"bucketCounts":["5","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601646873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601646873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601646873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":326},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601646873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601648109960919","count":"6","sum":3930425,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":366875,"max":956317},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601648109960919","count":"14","sum":69506749,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","14"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1011875,"max":9821180}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601648109964539","count":"14","sum":28,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601648109967069","count":"14","sum":168566,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","6","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":3253,"max":25690},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601648109967069","count":"1","sum":652,"bucketCounts":["0","0","0","0","0","0","0","0","0","1","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":652,"max":652}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601648109968929","count":"6","sum":0,"bucketCounts":["6","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601648109968929","count":"14","sum":14,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601648109969999","count":"6","sum":1,"bucketCounts":["5","1","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601648109969999","count":"14","sum":14,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601648109971059","count":"2","sum":2296338,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":267257,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601648109971059","count":"7","sum":1797094,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","7"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":101298,"max":398914}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601648109974469","count":"7","sum":84033,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","3","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":3245,"max":25674},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601648109974469","count":"2","sum":15247,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","1"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13316}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601648109976309","count":"2","sum":4,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601648109976309","count":"7","sum":14,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601648109977659","count":"2","sum":2,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601648109977659","count":"7","sum":7,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601648109979059","count":"2","sum":2,"bucketCounts":["0","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601648109979059","count":"7","sum":7,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601648109933599","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601648109948269","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601648109943109","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601648109948989","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601648109944399","asInt":"10000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601648109938069","asInt":"1000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601648109990878","asInt":"267"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601648109990878","asInt":"267"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601648109992078","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601648109992078","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601648109993518","asInt":"382"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601648109993518","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601648109993518","asInt":"382"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601648109994458","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601648109994458","asInt":"382"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601648109994458","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601648109997728","asInt":"13"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601648109998348","count":"13","sum":649,"bucketCounts":["4","3","2","2","0","1","1","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":5,"max":256}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601648110001258","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601648110002738","asInt":"267"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601648110003678","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601648110004378","asInt":"60"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601648110004378","asInt":"322"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601648110005108","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601648110005108","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601648110006538","asDouble":18.763811093}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601648109766732","asInt":"61934712"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601648110008608","asInt":"144697720"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601648109770622","asInt":"83616024"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601648110009638","asDouble":0.36}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601648109929079","asInt":"154664960"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601649465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601649465000000","asDouble":0.00071733}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601649465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601649465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601649465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601650439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601650439000000","asDouble":0.000614911}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601650439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601650439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601650439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":15205624}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":9428992}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":46800}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":33}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601651616000000","asDouble":1}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":190473}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":3967244}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601636.42}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":8809280}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":20}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":418}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":17522688}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601650}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":15205624}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":9}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0.13}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":101531}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601648.1501963}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":450240}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":30}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":29018424}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":14}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":1},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":9}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":1619}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0.004840212}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1933312}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":1619},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":17}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":88942}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":1229}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1229}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":35815424}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601650439}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601637670}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":30491840}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":522240}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":1229}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":44811}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":1},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","count":"9","sum":0,"bucketCounts":["9","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601651616000000","count":"1","sum":7355,"bucketCounts":["0","0","1","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":9}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":418}]}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":74186752}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1229}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","count":"6","sum":0.000991356,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000075919},{"quantile":0.5,"value":0.000213016},{"quantile":0.75,"value":0.000245897},{"quantile":1,"value":0.000255906}]}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":823234560}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":52557064}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":18292736}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":38400}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":0}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1048576}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601650}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1700601637}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1462764}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":1933312}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","count":"9","sum":0.013840854,"bucketCounts":["9","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601651616000000","count":"1","sum":0.003447061,"bucketCounts":["1","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601651616000000","asDouble":1}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":10000}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000"}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601651616000000"}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651616000000","asDouble":418}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":398}]}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"18","sum":36,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"3","sum":29147,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"9","sum":99427,"bucketCounts":["0","0","0","0","0","0","0","0","1","0","0","0","3","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":0.003156585}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":329},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":329}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":6}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601651873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601651873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601651873000000","asDouble":0}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":700},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":700}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"3","sum":3,"bucketCounts":["0","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"9","sum":9,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":700},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"3","sum":1944,"bucketCounts":["0","0","0","0","0","0","0","0","0","3","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"18","sum":199498,"bucketCounts":["0","0","0","0","0","0","0","0","2","0","0","0","6","0","0","10"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"18","sum":18,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"10","sum":3,"bucketCounts":["7","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"3","sum":6,"bucketCounts":["0","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"9","sum":18,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0.41000000000000003}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":92397848}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"19","sum":1029,"bucketCounts":["7","3","3","3","0","1","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"3","sum":3,"bucketCounts":["0","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"9","sum":9,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601651873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601651873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601651873000000","asDouble":10000}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":160432128}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":68850024}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":155923360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":110},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":590}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":329}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"18","sum":18,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"10","sum":0,"bucketCounts":["10","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"3","sum":2603994,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"9","sum":2207618,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","9"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601651873000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":23.76368367}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","asDouble":19}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"18","sum":91439076,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","18"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601651873000000","count":"10","sum":5960076,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","10"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601651873000000","asDouble":1}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601653109777530","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601653109769120","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601653109781840","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601653109778630","asInt":"10000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601653109773380","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601653109782520","asInt":"1000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601653109800589","asInt":"443"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601653109800589","asInt":"443"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601653109803479","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601653109803479","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601653109812479","asInt":"756"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601653109812479","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601653109812479","asInt":"756"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601653109813529","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601653109813529","asInt":"756"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601653109813529","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601653109816489","asInt":"22"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601653109817289","count":"22","sum":1199,"bucketCounts":["7","3","4","4","1","1","2","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601653109821539","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601653109823269","asInt":"443"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601653109824159","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601653109824899","asInt":"110"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601653109824899","asInt":"646"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601653109825739","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601653109825739","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601653109827079","asDouble":28.763561170000003}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601653109606522","asInt":"71877648"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601653109828559","asInt":"165866296"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601653109612652","asInt":"96592152"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601653109829299","asDouble":0.45999999999999996}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601653109764700","asInt":"163315712"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601653109832929","count":"11","sum":6397930,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":362075,"max":956317},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601653109832929","count":"22","sum":112429687,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":894667,"max":10489560}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601653109834929","count":"22","sum":44,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601653109836289","count":"3","sum":1944,"bucketCounts":["0","0","0","0","0","0","0","0","0","3","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":652},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601653109836289","count":"22","sum":296726,"bucketCounts":["0","0","0","0","0","0","0","0","2","0","0","0","6","0","0","14"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":29456}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601653109838459","count":"11","sum":0,"bucketCounts":["11","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601653109838459","count":"22","sum":22,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601653109839619","count":"11","sum":3,"bucketCounts":["8","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601653109839619","count":"22","sum":22,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601653109840579","count":"12","sum":2840258,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","12"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":101298,"max":398914},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601653109840579","count":"3","sum":2603994,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":267257,"max":2029081}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601653109842409","count":"3","sum":29147,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601653109842409","count":"12","sum":148025,"bucketCounts":["0","0","0","0","0","0","0","0","1","0","0","1","3","0","0","7"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":27172}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601653109843969","count":"3","sum":6,"bucketCounts":["0","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601653109843969","count":"12","sum":24,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601653109845009","count":"3","sum":3,"bucketCounts":["0","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601653109845009","count":"12","sum":12,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601653109846429","count":"12","sum":12,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601653109846429","count":"3","sum":3,"bucketCounts":["0","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601654465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601654465000000","asDouble":0.000670451}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601654465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601654465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601654465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601655439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601655439000000","asDouble":0.000754069}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601655439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601655439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601655439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":108441}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":14}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":51749}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0.005656769}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":418}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":21137856}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":24387584}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":29018424}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":10000}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000"}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":538560}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1966080}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601655439}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":2},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601656616000000","asDouble":1}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601655}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":2857},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":532160}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","count":"14","sum":0.019636270999999997,"bucketCounts":["14","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601656616000000","count":"2","sum":0.006254911,"bucketCounts":["2","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":2857}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1048576}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","count":"6","sum":0.000991356,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000075919},{"quantile":0.5,"value":0.000213016},{"quantile":0.75,"value":0.000245897},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":21137856}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":30}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":35782656}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":14}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":20}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":823234560}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601637}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":2},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":14}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1966080}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","count":"14","sum":0,"bucketCounts":["14","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601656616000000","count":"2","sum":14844,"bucketCounts":["0","0","2","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601637}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":11395072}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601636.42}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000"}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":418}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":3915596}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":36424072}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":9273344}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0.15}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1}]}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601637670}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601656616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":23}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":33}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601655}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601656616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":418}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":8842216}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":231210}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":76546048}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":122769}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":38400}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":52557064}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1465156}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":46800}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":14}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1252}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":1700601648.1501963}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601656616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":0.003644209}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":1074},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":169607168}]}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"4","sum":43047,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"14","sum":162589,"bucketCounts":["0","0","0","0","0","0","0","0","2","0","0","1","3","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0.52}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"26","sum":26,"bucketCounts":["0","26","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"15","sum":0,"bucketCounts":["15","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"26","sum":26,"bucketCounts":["0","26","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"15","sum":5,"bucketCounts":["10","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":175521208}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"4","sum":4,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"14","sum":14,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"4","sum":2916650,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"14","sum":3221453,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","14"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601656873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601656873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601656873000000","asDouble":10000}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":502},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":502}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":502}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":1}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":1074},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":1074}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"28","sum":1576,"bucketCounts":["10","3","5","5","1","1","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":28}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"26","sum":52,"bucketCounts":["0","26","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601656873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601656873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601656873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601656873000000","asDouble":0}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"26","sum":134842807,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","26"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"15","sum":8502659,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","15"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":160},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":914}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"4","sum":8,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"14","sum":28,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":78504936}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":33.763858623000004}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"4","sum":4,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"14","sum":14,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":398}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601656873000000","asDouble":100786456}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"5","sum":3236,"bucketCounts":["0","0","0","0","0","0","0","0","0","5","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601656873000000","count":"26","sum":325994,"bucketCounts":["0","0","0","0","0","0","0","0","4","0","0","0","6","0","0","16"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601658110033918","asDouble":38.764129122}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601658109852321","asInt":"54655448"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601658110037678","asInt":"187511888"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601658109855601","asInt":"109437208"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601658110039028","asDouble":0.6100000000000001}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601658110016378","asInt":"174063616"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601658110044668","count":"16","sum":8868004,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","16"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":362075,"max":956317},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601658110044668","count":"30","sum":155435803,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","30"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":894667,"max":10489560}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601658110046898","count":"30","sum":60,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601658110048728","count":"30","sum":412770,"bucketCounts":["0","0","0","0","0","0","0","0","4","0","0","0","6","0","0","20"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":29456},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601658110048728","count":"5","sum":3236,"bucketCounts":["0","0","0","0","0","0","0","0","0","5","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":652}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601658110050068","count":"16","sum":0,"bucketCounts":["16","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601658110050068","count":"30","sum":30,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601658110051288","count":"16","sum":5,"bucketCounts":["11","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601658110051288","count":"30","sum":30,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601658110052358","count":"4","sum":2916650,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":267257,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601658110052358","count":"17","sum":3812435,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","17"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":398914}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601658110053958","count":"4","sum":43047,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601658110053958","count":"17","sum":205963,"bucketCounts":["0","0","0","0","0","0","0","0","2","0","0","2","3","0","0","10"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":27172}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601658110055158","count":"4","sum":8,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601658110055158","count":"17","sum":34,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601658110056048","count":"4","sum":4,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601658110056048","count":"17","sum":17,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601658110056958","count":"4","sum":4,"bucketCounts":["0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601658110056958","count":"17","sum":17,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601658110020078","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601658110030958","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601658110026618","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601658110022678","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601658110031598","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601658110027548","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601658110060948","asInt":"604"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601658110060948","asInt":"604"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601658110061818","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601658110061818","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601658110063238","asInt":"1130"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601658110063238","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601658110063238","asInt":"1130"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601658110063948","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601658110063948","asInt":"1130"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601658110063948","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601658110066368","asInt":"31"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601658110066868","count":"31","sum":1734,"bucketCounts":["10","3","6","7","1","1","3","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601658110070028","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601658110070808","asInt":"604"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601658110071698","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601658110072328","asInt":"160"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601658110072328","asInt":"970"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601658110072838","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601658110072838","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601659465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601659465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601659465000000","asDouble":0.000624761}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601659465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601659465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601660439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601660439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601660439000000","asDouble":0.00065403}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601660439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601660439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":418}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":187494}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":8883320}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":3},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":4095}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":824545280}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":30806408}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1467860}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":10000}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601660}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":35}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":15933120}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0.003898694}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1048576}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":14}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":5051948}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":28}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":418}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000"}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":669120}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":4095},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":19136512}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601658.2016292}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":38400}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601661616000000","asDouble":1}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":15933120}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","count":"7","sum":0.001051265,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000059909},{"quantile":0.5,"value":0.000154358},{"quantile":0.75,"value":0.000245897},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601661616000000","asDouble":1}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":20}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","count":"19","sum":0.026902977999999994,"bucketCounts":["19","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601661616000000","count":"3","sum":0.009576604,"bucketCounts":["3","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":2031616}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":89485}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":39911424}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601660}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":19}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":19}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":33}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":9035776}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":3},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":19}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","count":"19","sum":0,"bucketCounts":["19","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601661616000000","count":"3","sum":22341,"bucketCounts":["0","0","3","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601660439}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":276979}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":81002496}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601661616000000","asDouble":0}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":510240}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":2031616}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0.18}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601637670}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":418}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":20774912}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":58062088}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000"}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":43420576}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":46800}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":1700601636.42}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661616000000","asDouble":57862}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601661616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":662},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":662}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"34","sum":34,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"20","sum":0,"bucketCounts":["20","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":1448},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":662}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"34","sum":175725264,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","34"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"20","sum":11108772,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","20"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":201847224}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"37","sum":2110,"bucketCounts":["13","3","7","8","1","1","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"5","sum":10,"bucketCounts":["0","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"19","sum":38,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":63627392}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"5","sum":56947,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"19","sum":220441,"bucketCounts":["0","0","0","0","0","0","0","0","3","0","0","2","3","0","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601661873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":0.002468534}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601661873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601661873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601661873000000","asDouble":10000}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":210},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":1238}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"34","sum":68,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"6","sum":3888,"bucketCounts":["0","0","0","0","0","0","0","0","0","6","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"34","sum":441862,"bucketCounts":["0","0","0","0","0","0","0","0","6","0","0","0","6","0","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":109699352}]}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"5","sum":3112967,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"19","sum":4092022,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","19"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"5","sum":5,"bucketCounts":["0","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"19","sum":19,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"5","sum":5,"bucketCounts":["0","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"19","sum":19,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":1448},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":1448}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":37}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"34","sum":34,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","count":"20","sum":6,"bucketCounts":["14","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":0.6799999999999999}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":175898624}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601661873000000","asDouble":43.764610704999996}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":1}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":398}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601661873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601661873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601661873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601661873000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601663109526429","asInt":"757"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601663109528269","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601663109529359","asInt":"210"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601663109529359","asInt":"1294"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601663109530929","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601663109530929","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601663109532539","asDouble":48.764141405000004}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601663109388991","asInt":"67298448"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601663109535709","asInt":"214490224"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601663109392291","asInt":"109699352"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601663109536639","asDouble":0.72}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601663109508169","asInt":"176947200"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601663109541149","count":"21","sum":11898070,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","21"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":362075,"max":956317},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601663109541149","count":"38","sum":203056272,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","38"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":894667,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601663109543339","count":"38","sum":76,"bucketCounts":["0","38","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601663109544909","count":"6","sum":3888,"bucketCounts":["0","0","0","0","0","0","0","0","0","6","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":652},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601663109544909","count":"38","sum":522934,"bucketCounts":["0","0","0","0","0","0","0","0","6","0","0","0","6","0","0","26"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":29456}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601663109546059","count":"21","sum":0,"bucketCounts":["21","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601663109546059","count":"38","sum":38,"bucketCounts":["0","38","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601663109547238","count":"21","sum":6,"bucketCounts":["15","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601663109547238","count":"38","sum":38,"bucketCounts":["0","38","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601663109548088","count":"5","sum":3112967,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":196317,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601663109548088","count":"22","sum":4522006,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":398914}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601663109549668","count":"5","sum":56947,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601663109549668","count":"22","sum":260963,"bucketCounts":["0","0","0","0","0","0","0","0","3","0","0","3","3","0","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":27172}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601663109550958","count":"5","sum":10,"bucketCounts":["0","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601663109550958","count":"22","sum":44,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601663109551878","count":"5","sum":5,"bucketCounts":["0","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601663109551878","count":"22","sum":22,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601663109552718","count":"5","sum":5,"bucketCounts":["0","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601663109552718","count":"22","sum":22,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601663109512229","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601663109522769","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601663109519169","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601663109515609","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601663109523299","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601663109520119","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601663109557318","asInt":"757"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601663109557318","asInt":"757"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601663109557958","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601663109557958","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601663109558748","asInt":"1504"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601663109558748","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601663109558748","asInt":"1504"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601663109559288","asInt":"1504"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601663109559288","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601663109559288","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601663109561238","asInt":"40"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601663109561548","count":"40","sum":2261,"bucketCounts":["13","3","8","10","1","1","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601663109564348","asInt":"6"}],"aggregationTemporality":2}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601664465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601664465000000","asDouble":0.000616201}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601664465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601664465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601664465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601665439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601665439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601665439000000","asDouble":0.0006541}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601665439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601665439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000"}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":581920}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":418}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":4},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":418}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","count":"7","sum":0.001051265,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000059909},{"quantile":0.5,"value":0.000154358},{"quantile":0.75,"value":0.000245897},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":4},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":24}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":8891560}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601666616000000","asDouble":1}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000"}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601637670}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","count":"24","sum":0.031817556999999996,"bucketCounts":["24","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601666616000000","count":"4","sum":0.011753723,"bucketCounts":["4","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":64163}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":48504960}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":10000}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":8937472}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601666616000000","asDouble":1}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601665}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601637}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":33}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0.003689138}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":824545280}]}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":21017504}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":15605760}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":46800}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":58062088}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1048576}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":2064384}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601636.42}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":317465}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":669120}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":2064384}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0.19}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":24}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601666616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":38400}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601665}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":20}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":418}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":24272896}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601658.2016292}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601665439}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":123064}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":21017504}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":39878656}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":81264640}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1469372}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":194401}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":14}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":5333}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":5333},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":24}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":5042196}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":35}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":30806408}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","count":"24","sum":0,"bucketCounts":["24","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601666616000000","count":"4","sum":29842,"bucketCounts":["0","0","4","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":33}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601666616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666616000000","asDouble":1252}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"6","sum":12,"bucketCounts":["0","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"24","sum":48,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":177995776}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":224770800}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":53.76360519}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"6","sum":3262615,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"24","sum":4780773,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","24"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":816},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":816}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601666873000000","asDouble":1}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":0.002961137}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601666873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601666873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601666873000000","asDouble":10000}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":109699352}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"46","sum":2638,"bucketCounts":["16","3","9","11","1","1","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"6","sum":6,"bucketCounts":["0","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"24","sum":24,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":6}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":1822},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"42","sum":42,"bucketCounts":["0","42","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"25","sum":9,"bucketCounts":["16","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":1822},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":1822}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":73907968}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":260},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":1562}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"42","sum":220762239,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","42"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"25","sum":13844492,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","25"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"6","sum":6,"bucketCounts":["0","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"24","sum":24,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":46}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"42","sum":84,"bucketCounts":["0","42","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"42","sum":42,"bucketCounts":["0","42","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"25","sum":0,"bucketCounts":["25","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"6","sum":70847,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"24","sum":275527,"bucketCounts":["0","0","0","0","0","0","0","0","4","0","0","3","3","0","0","14"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601666873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601666873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601666873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601666873000000","asDouble":0}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":0.78}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","asDouble":816}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"9","sum":19080,"bucketCounts":["0","0","0","0","0","0","0","0","0","8","0","0","0","0","0","1"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601666873000000","count":"42","sum":552202,"bucketCounts":["0","0","0","0","0","0","0","0","8","0","0","0","6","0","0","28"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601668110030953","asDouble":58.764058894}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601668109866385","asInt":"76854672"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601668110035193","asInt":"234327024"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601668109878185","asInt":"109699352"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601668110036753","asDouble":0.83}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601668110010863","asInt":"177995776"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601668110041673","count":"26","sum":14237856,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","26"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":331365,"max":956317},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601668110041673","count":"46","sum":240898531,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","46"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":630041,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601668110044533","count":"46","sum":92,"bucketCounts":["0","46","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601668110046503","count":"46","sum":651774,"bucketCounts":["0","0","0","0","0","0","0","0","8","0","0","0","6","0","0","32"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":29456},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601668110046503","count":"9","sum":19080,"bucketCounts":["0","0","0","0","0","0","0","0","0","8","0","0","0","0","0","1"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601668110048113","count":"26","sum":0,"bucketCounts":["26","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601668110048113","count":"46","sum":46,"bucketCounts":["0","46","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601668110049293","count":"46","sum":46,"bucketCounts":["0","46","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601668110049293","count":"26","sum":9,"bucketCounts":["17","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601668110050363","count":"6","sum":3262615,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601668110050363","count":"27","sum":5343665,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","27"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":398914}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601668110052173","count":"6","sum":70847,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601668110052173","count":"27","sum":325295,"bucketCounts":["0","0","0","0","0","0","0","0","4","0","0","4","3","0","0","16"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":27172}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601668110053933","count":"6","sum":12,"bucketCounts":["0","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601668110053933","count":"27","sum":54,"bucketCounts":["0","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601668110062772","count":"6","sum":6,"bucketCounts":["0","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601668110062772","count":"27","sum":27,"bucketCounts":["0","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601668110063822","count":"6","sum":6,"bucketCounts":["0","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601668110063822","count":"27","sum":27,"bucketCounts":["0","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601668110015093","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601668110028143","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601668110023563","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601668110019143","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601668110028813","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601668110024783","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601668110068572","asInt":"931"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601668110068572","asInt":"931"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601668110069282","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601668110069282","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601668110070612","asInt":"1878"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601668110070612","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601668110070612","asInt":"1878"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601668110071372","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601668110071372","asInt":"1878"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601668110071372","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601668110073492","asInt":"49"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601668110073892","count":"49","sum":2809,"bucketCounts":["16","3","10","13","1","1","5","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601668110076422","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601668110077502","asInt":"931"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601668110078392","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601668110078912","asInt":"1618"},{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601668110078912","asInt":"260"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601668110079492","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601668110079492","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601669465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601669465000000","asDouble":0.000579332}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601669465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601669465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601669465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601670439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601670439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601670439000000","asDouble":0.00066123}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601670439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601670439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":20}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601670439}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":10000}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":39}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":418}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":46800}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1469956}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":29}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1048576}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":35}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":58062088}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","count":"29","sum":0.037789382,"bucketCounts":["29","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601671616000000","count":"5","sum":0.013744545,"bucketCounts":["5","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":5035444}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":6571},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601637}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0.005285424}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":418}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":30806408}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601658.2016292}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000"}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601671616000000","asDouble":1}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":156422}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":39845888}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0.2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":26796344}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":824545280}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000"}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":357854}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","count":"29","sum":0,"bucketCounts":["29","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601671616000000","count":"5","sum":37341,"bucketCounts":["0","0","5","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601670}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":29}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":2097152}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601670}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":5},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","count":"7","sum":0.001051265,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000059909},{"quantile":0.5,"value":0.000154358},{"quantile":0.75,"value":0.000245897},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":38400}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":82051072}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601671616000000","asDouble":1}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":201432}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1252}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601637670}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":70464}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":54283800}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":6571}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":663840}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":30998528}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":669120}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":26796344}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":8897728}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":8847360}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":14}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":5},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":29}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":33}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":2097152}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":418}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":8478720}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":1700601636.42}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671616000000","asDouble":0}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601671616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":243711984}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":310},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":1886}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"50","sum":260151044,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","50"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"30","sum":16138680,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","30"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"12","sum":34272,"bucketCounts":["0","0","0","0","0","0","0","0","0","10","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"50","sum":680866,"bucketCounts":["0","0","0","0","0","0","0","0","10","0","0","0","6","0","0","34"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"7","sum":3429933,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","7"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"29","sum":5648580,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","29"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"7","sum":14,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"29","sum":58,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601671873000000","asDouble":1}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601671873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601671873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601671873000000","asDouble":10000}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601671873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601671873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601671873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":2196},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0.87}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"55","sum":3185,"bucketCounts":["19","3","11","14","1","1","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":989},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":989}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":55}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"7","sum":84747,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"29","sum":339773,"bucketCounts":["0","0","0","0","0","0","0","0","5","0","0","4","3","0","0","17"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":0.004362457}]}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"50","sum":50,"bucketCounts":["0","50","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"30","sum":0,"bucketCounts":["30","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"50","sum":50,"bucketCounts":["0","50","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"30","sum":12,"bucketCounts":["18","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":2196},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":2196}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":179306496}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":63.764286772999995}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":989}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"50","sum":100,"bucketCounts":["0","50","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":398}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":83292928}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":109699352}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":398}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"7","sum":7,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"29","sum":29,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"7","sum":7,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601671873000000","count":"29","sum":29,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601671873000000","asDouble":398}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601673109464648","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601673109477178","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601673109472878","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601673109468148","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601673109477738","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601673109473918","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601673109484468","asInt":"1094"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601673109484468","asInt":"1094"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601673109486248","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601673109486248","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601673109487618","asInt":"2252"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601673109487618","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601673109487618","asInt":"2252"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601673109488608","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601673109488608","asInt":"2252"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601673109488608","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601673109490718","asInt":"58"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601673109491788","count":"58","sum":3346,"bucketCounts":["19","3","12","16","1","1","6","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601673109495228","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601673109496198","asInt":"1094"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601673109497308","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601673109497998","asInt":"310"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601673109497998","asInt":"1942"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601673109498648","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601673109498648","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601673109500078","asDouble":68.763753039}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601673109315371","asInt":"86997944"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601673109501488","asInt":"253855256"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601673109320300","asInt":"109699352"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601673109502458","asDouble":0.92}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601673109460618","asInt":"181927936"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601673109505918","count":"31","sum":16550614,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","31"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":323206,"max":956317},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601673109505918","count":"54","sum":281770214,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","54"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601673109508078","count":"54","sum":108,"bucketCounts":["0","54","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601673109509088","count":"54","sum":769670,"bucketCounts":["0","0","0","0","0","0","0","0","10","0","0","0","6","0","0","38"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":29456},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601673109509088","count":"12","sum":34272,"bucketCounts":["0","0","0","0","0","0","0","0","0","10","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601673109510248","count":"31","sum":0,"bucketCounts":["31","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601673109510248","count":"54","sum":54,"bucketCounts":["0","54","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601673109511248","count":"31","sum":12,"bucketCounts":["19","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601673109511248","count":"54","sum":54,"bucketCounts":["0","54","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601673109512168","count":"7","sum":3429933,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","7"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601673109512168","count":"32","sum":7783450,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","32"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601673109513868","count":"7","sum":84747,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601673109513868","count":"32","sum":384161,"bucketCounts":["0","0","0","0","0","0","0","0","5","0","0","5","3","0","0","19"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":27172}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601673109515328","count":"32","sum":64,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601673109515328","count":"7","sum":14,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601673109517018","count":"7","sum":7,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601673109517018","count":"32","sum":32,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601673109518128","count":"7","sum":7,"bucketCounts":["0","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601673109518128","count":"32","sum":32,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601674465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601674465000000","asDouble":0.00070008}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601674465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601674465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601674465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601675439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601675439000000","asDouble":0.000599311}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601675439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601675439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601675439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":24059904}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":62256392}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":14}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601636.42}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","count":"34","sum":0.04261569099999999,"bucketCounts":["34","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601676616000000","count":"6","sum":0.017117857,"bucketCounts":["6","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1470348}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":61480616}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":7809},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0.004813311}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":10000}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":20}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1}]}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601676616000000","asDouble":1}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1048576}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601676616000000","asDouble":1}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":7809}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":545440}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":824545280}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":113300}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":34}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":2064384}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":290596}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":6},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":34}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1252}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":33}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601675}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601675}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601637670}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","count":"34","sum":0,"bucketCounts":["34","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601676616000000","count":"6","sum":44842,"bucketCounts":["0","0","6","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601671.6189582}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000"}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":10584064}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","count":"8","sum":0.001119594,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000068329},{"quantile":0.5,"value":0.000154358},{"quantile":0.75,"value":0.000245897},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":83886080}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":8899776}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":418}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0.23}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":44}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":38400}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":418}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000"}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":6},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":44072960}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":34}]}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":35}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":46800}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":31446440}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601637}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":2064384}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":5016684}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":76756}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":20370352}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":418}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":20013056}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":685440}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":403896}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601676616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":1700601675439}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676616000000","asDouble":20370352}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601676873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601676873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601676873000000","asDouble":0}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":1153},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":1153}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":264069040}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":6}]}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"58","sum":58,"bucketCounts":["0","58","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"35","sum":0,"bucketCounts":["35","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":398}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"58","sum":301362623,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","58"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"35","sum":18702735,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","35"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"8","sum":16,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"34","sum":68,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":398}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":187695104}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":118153496}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":73.763547432}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":1153}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"8","sum":98647,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","7"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"34","sum":398725,"bucketCounts":["0","0","0","0","0","0","0","0","6","0","0","5","3","0","0","20"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":398}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":0.003999112}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0.97}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601676873000000","asDouble":93506712}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"64","sum":3723,"bucketCounts":["22","3","13","17","1","1","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":64}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"14","sum":35564,"bucketCounts":["0","0","0","0","0","0","0","0","0","12","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"58","sum":798938,"bucketCounts":["0","0","0","0","0","0","0","0","12","0","0","0","6","0","0","40"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"58","sum":58,"bucketCounts":["0","58","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"35","sum":14,"bucketCounts":["21","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":2570},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":2570},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":2570}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":360},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":2210}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"58","sum":116,"bucketCounts":["0","58","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601676873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601676873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601676873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601676873000000","asDouble":10000}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"8","sum":3612320,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"34","sum":8037426,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","34"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"8","sum":8,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"34","sum":34,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"8","sum":8,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601676873000000","count":"34","sum":34,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601678110109163","count":"36","sum":19226317,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","36"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":323206,"max":956317},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601678110109163","count":"62","sum":321990186,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","62"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601678110111973","count":"62","sum":124,"bucketCounts":["0","62","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601678110117823","count":"62","sum":879594,"bucketCounts":["0","0","0","0","0","0","0","0","12","0","0","0","6","0","0","44"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":29456},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601678110117823","count":"14","sum":35564,"bucketCounts":["0","0","0","0","0","0","0","0","0","12","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601678110119223","count":"62","sum":62,"bucketCounts":["0","62","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601678110119223","count":"36","sum":0,"bucketCounts":["36","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601678110120783","count":"36","sum":14,"bucketCounts":["22","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601678110120783","count":"62","sum":62,"bucketCounts":["0","62","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601678110121863","count":"8","sum":3612320,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601678110121863","count":"37","sum":8537020,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","37"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601678110123743","count":"8","sum":98647,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","7"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601678110123743","count":"37","sum":439037,"bucketCounts":["0","0","0","0","0","0","0","0","6","0","0","6","3","0","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":27172}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601678110125183","count":"8","sum":16,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601678110125183","count":"37","sum":74,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601678110126123","count":"8","sum":8,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601678110126123","count":"37","sum":37,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601678110129573","count":"8","sum":8,"bucketCounts":["0","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601678110129573","count":"37","sum":37,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601678110087543","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601678110100983","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601678110096183","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601678110101673","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601678110097423","asInt":"10000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601678110092033","asInt":"1000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601678110136793","asInt":"1243"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601678110136793","asInt":"1243"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601678110137723","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601678110137723","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601678110139333","asInt":"2626"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601678110139333","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601678110139333","asInt":"2626"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601678110140142","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601678110140142","asInt":"2626"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601678110140142","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601678110142572","asInt":"67"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601678110143122","count":"67","sum":3869,"bucketCounts":["22","3","14","19","1","1","7","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601678110148002","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601678110149082","asInt":"1243"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601678110149972","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601678110150552","asInt":"360"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601678110150552","asInt":"2266"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601678110151612","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601678110151612","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601678110152772","asDouble":78.764164096}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601678109942505","asInt":"96349704"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601678110154562","asInt":"273420800"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601678109946075","asInt":"122347800"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601678110155232","asDouble":1.02}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601678110083273","asInt":"190316544"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601679465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601679465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601679465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601679465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601679465000000","asDouble":0.00066087}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601680439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601680439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601680439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601680439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601680439000000","asDouble":0.000604492}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601681616000000","asDouble":1}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":637600}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":10584064}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000"}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":26267048}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":83886080}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0.004526044}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":418}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":39}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601671.6189582}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1471356}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":46800}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":5275772}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":2064384}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":418}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":38400}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":9047},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":39}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0.25}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601680439}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":44072960}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1048576}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":83160}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":9047}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000"}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":685440}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":2064384}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":62518536}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601636.42}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":34}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":443935}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":14}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":67377312}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":26267048}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":30351360}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":35}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","count":"39","sum":0,"bucketCounts":["39","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601681616000000","count":"7","sum":52339,"bucketCounts":["0","0","7","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601680}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","count":"8","sum":0.001119594,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000068329},{"quantile":0.5,"value":0.000154358},{"quantile":0.75,"value":0.000245897},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":8901824}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":10000}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":297189}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":418}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":31446440}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":13721600}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":146746}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":824807424}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","count":"39","sum":0.047863485,"bucketCounts":["39","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601681616000000","count":"7","sum":0.019777029,"bucketCounts":["7","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601680}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601681616000000","asDouble":1}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":20}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":7},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":49}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":7},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":39}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601681616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681616000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"9","sum":9,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"39","sum":39,"bucketCounts":["0","39","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":197394432}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":130736408}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":83.764140513}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"73","sum":4245,"bucketCounts":["25","3","15","20","1","1","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":6}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":1.1099999999999999}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"66","sum":132,"bucketCounts":["0","66","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":1}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":283559848}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":410},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":2534}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"9","sum":9,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"39","sum":39,"bucketCounts":["0","39","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601681873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601681873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601681873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601681873000000","asDouble":10000}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601681873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601681873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601681873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":2944},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":1301}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"66","sum":66,"bucketCounts":["0","66","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"40","sum":16,"bucketCounts":["24","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"9","sum":3836917,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","9"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"39","sum":8862085,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","39"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":56238944}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":73}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"66","sum":344628181,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","66"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"40","sum":21530874,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","40"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"9","sum":18,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"39","sum":78,"bucketCounts":["0","39","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":1301},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":1301}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":0.002828329}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":2944},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","asDouble":2944}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"66","sum":66,"bucketCounts":["0","66","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"40","sum":0,"bucketCounts":["40","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"16","sum":36856,"bucketCounts":["0","0","0","0","0","0","0","0","0","14","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"66","sum":908686,"bucketCounts":["0","0","0","0","0","0","0","0","14","0","0","0","6","0","0","46"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"9","sum":112547,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601681873000000","count":"39","sum":453515,"bucketCounts":["0","0","0","0","0","0","0","0","7","0","0","6","3","0","0","23"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601681873000000","asDouble":398}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601683110034686","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601683110030586","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601683110022717","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601683110026577","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601683110035336","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601683110031596","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601683110045306","asInt":"1397"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601683110045306","asInt":"1397"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601683110047366","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601683110047366","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601683110048506","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601683110048506","asInt":"3000"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601683110048506","asInt":"3000"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601683110049356","asInt":"3000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601683110049356","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601683110049356","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601683110052116","asInt":"76"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601683110052856","count":"76","sum":4397,"bucketCounts":["25","3","16","22","1","1","8","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601683110056146","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601683110057946","asInt":"1397"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601683110058836","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601683110059556","asInt":"410"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601683110059556","asInt":"2590"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601683110060386","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601683110060386","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601683110061576","asDouble":88.764067127}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601683109848429","asInt":"59138984"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601683110062916","asInt":"293755944"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601683109853369","asInt":"130736408"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601683110063676","asDouble":1.2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601683110018417","asInt":"197918720"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601683110068436","count":"41","sum":21902389,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","41"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":323206,"max":1121404},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601683110068436","count":"70","sum":363625468,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","70"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601683110070246","count":"70","sum":140,"bucketCounts":["0","70","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601683110071306","count":"70","sum":993386,"bucketCounts":["0","0","0","0","0","0","0","0","14","0","0","0","6","0","0","50"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":29456},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601683110071306","count":"16","sum":36856,"bucketCounts":["0","0","0","0","0","0","0","0","0","14","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601683110072606","count":"41","sum":0,"bucketCounts":["41","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601683110072606","count":"70","sum":70,"bucketCounts":["0","70","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601683110073686","count":"41","sum":16,"bucketCounts":["25","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601683110073686","count":"70","sum":70,"bucketCounts":["0","70","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601683110074986","count":"42","sum":9279359,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","42"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601683110074986","count":"9","sum":3836917,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","9"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601683110076396","count":"42","sum":495849,"bucketCounts":["0","0","0","0","0","0","0","0","7","0","0","7","3","0","0","25"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":27172},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601683110076396","count":"9","sum":112547,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601683110077816","count":"9","sum":18,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601683110077816","count":"42","sum":84,"bucketCounts":["0","42","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601683110079536","count":"9","sum":9,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601683110079536","count":"42","sum":42,"bucketCounts":["0","42","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601683110080876","count":"9","sum":9,"bucketCounts":["0","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601683110080876","count":"42","sum":42,"bucketCounts":["0","42","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601684465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601684465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601684465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601684465000000","asDouble":0.000611831}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601684465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601685439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601685439000000","asDouble":0.000610731}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601685439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601685439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601685439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":54}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":418}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","count":"9","sum":0.001179444,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000059909},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000213016},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":19028968}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":8901824}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601636.42}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000"}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","count":"44","sum":0.055629073999999994,"bucketCounts":["44","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601686616000000","count":"8","sum":0.022070215,"bucketCounts":["8","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":379645}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":44072960}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":418}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":549440}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":44}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":33}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":22487040}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":38400}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":8},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":62518536}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":36}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601685439}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":10285}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601686616000000","asDouble":1}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":2064384}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":84148224}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":89475}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","count":"44","sum":0,"bucketCounts":["44","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601686616000000","count":"8","sum":59836,"bucketCounts":["0","0","8","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601685}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":5274620}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":20}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":44}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":489482}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":19028968}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":14}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601686616000000","asDouble":1}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":685440}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":31365896}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":2064384}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":10108928}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":8},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":44}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":10285},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601637670}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":73659024}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":824807424}]}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":46800}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":10000}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0.00349482}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":109837}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601681.7157576}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1472508}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0.29}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":21585920}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1048576}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000"}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":1700601685}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686616000000","asDouble":418}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601686616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":1456}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"74","sum":148,"bucketCounts":["0","74","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601686873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601686873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601686873000000","asDouble":0}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"74","sum":386145963,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","74"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"45","sum":23950391,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","45"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":1456},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":1456}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"74","sum":74,"bucketCounts":["0","74","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"45","sum":18,"bucketCounts":["27","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":1}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":398}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":1.24}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":3318},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":3318},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":3318}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":198180864}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":303304640}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":93.76406895400001}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"10","sum":4033754,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","10"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"44","sum":9549466,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","44"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601686873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601686873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601686873000000","asDouble":10000}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":460},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":2858}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"74","sum":74,"bucketCounts":["0","74","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"45","sum":0,"bucketCounts":["45","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"18","sum":38148,"bucketCounts":["0","0","0","0","0","0","0","0","0","16","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"74","sum":1022654,"bucketCounts":["0","0","0","0","0","0","0","0","16","0","0","0","6","0","0","52"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"10","sum":10,"bucketCounts":["0","10","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"44","sum":44,"bucketCounts":["0","44","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"82","sum":4774,"bucketCounts":["28","3","17","23","1","1","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":6}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"10","sum":10,"bucketCounts":["0","10","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"44","sum":44,"bucketCounts":["0","44","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601686873000000","asDouble":1}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":65787640}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","asDouble":82}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"10","sum":126447,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","9"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"44","sum":510413,"bucketCounts":["0","0","0","0","0","0","0","0","8","0","0","7","3","0","0","26"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"10","sum":20,"bucketCounts":["0","10","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601686873000000","count":"44","sum":88,"bucketCounts":["0","44","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":0.003059246}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601686873000000","asDouble":130736408}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601688109296559","asInt":"1561"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601688109296559","asInt":"1561"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601688109298869","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601688109298869","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601688109300709","asInt":"3374"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601688109300709","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601688109300709","asInt":"3374"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601688109302409","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601688109302409","asInt":"3374"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601688109302409","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601688109304869","asInt":"85"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601688109305619","count":"85","sum":4935,"bucketCounts":["28","3","18","25","1","1","9","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601688109311529","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601688109315988","asInt":"1561"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601688109317198","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601688109317838","asInt":"460"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601688109317838","asInt":"2914"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601688109319458","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601688109319458","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601688109321318","asDouble":98.763360106}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601688109136151","asInt":"69484872"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601688109324568","asInt":"313650528"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601688109139831","asInt":"130736408"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601688109326368","asDouble":1.2799999999999998}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601688109278079","asInt":"198180864"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601688109330608","count":"46","sum":24272187,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","46"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":321796,"max":1121404},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601688109330608","count":"78","sum":408809275,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","78"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601688109332598","count":"78","sum":156,"bucketCounts":["0","78","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601688109333728","count":"78","sum":1108846,"bucketCounts":["0","0","0","0","0","0","0","0","16","0","0","0","6","0","0","56"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":29456},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601688109333728","count":"18","sum":38148,"bucketCounts":["0","0","0","0","0","0","0","0","0","16","0","0","0","0","0","2"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601688109334998","count":"46","sum":0,"bucketCounts":["46","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601688109334998","count":"78","sum":78,"bucketCounts":["0","78","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601688109335828","count":"46","sum":18,"bucketCounts":["28","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601688109335828","count":"78","sum":78,"bucketCounts":["0","78","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601688109336988","count":"10","sum":4033754,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","10"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601688109336988","count":"47","sum":10035638,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","47"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601688109338748","count":"10","sum":126447,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","9"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601688109338748","count":"47","sum":553495,"bucketCounts":["0","0","0","0","0","0","0","0","8","0","0","7","4","0","0","28"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":27172}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601688109340508","count":"47","sum":94,"bucketCounts":["0","47","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601688109340508","count":"10","sum":20,"bucketCounts":["0","10","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601688109341928","count":"47","sum":47,"bucketCounts":["0","47","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601688109341928","count":"10","sum":10,"bucketCounts":["0","10","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601688109342868","count":"10","sum":10,"bucketCounts":["0","10","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601688109342868","count":"47","sum":47,"bucketCounts":["0","47","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601688109281569","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601688109292829","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601688109288759","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601688109284399","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601688109293389","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601688109289679","asInt":"10000"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601689465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601689465000000","asDouble":0.000613031}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601689465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601689465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601689465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601690439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601690439000000","asDouble":0.000623111}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601690439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601690439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601690439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":824807424}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601690}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":15171584}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":28868608}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":685440}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":418}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":418}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":2097152}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":14}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":5274252}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":9},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":20}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":33}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":143028}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":44040192}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":38400}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000"}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":49}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":11523},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":59}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":9658368}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","count":"49","sum":0,"bucketCounts":["49","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601691616000000","count":"9","sum":67336,"bucketCounts":["0","0","9","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0.004376897}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","count":"49","sum":0.060110357,"bucketCounts":["49","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601691616000000","count":"9","sum":0.023868379000000002,"bucketCounts":["9","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000"}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","count":"9","sum":0.001179444,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.000059909},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000213016},{"quantile":1,"value":0.000255906}]}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1048576}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":386575}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":529603}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":46800}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":36}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601636.42}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":24921632}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601681.7157576}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0.3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":418}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":11523}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":95958}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601691616000000","asDouble":1}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":9},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":49}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":10000}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601690439}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":2097152}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":24921632}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":8901824}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601691616000000","asDouble":1}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":62518536}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":621120}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601690}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1700601637}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":79551688}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":1472876}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":84410368}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601691616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":49}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":31365896}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691616000000","asDouble":18446744073709552000}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":6}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":3692},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":76034688}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"82","sum":432082000,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","82"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"50","sum":26274928,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","50"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"82","sum":82,"bucketCounts":["0","82","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"50","sum":21,"bucketCounts":["29","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"11","sum":140347,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","10"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"49","sum":568434,"bucketCounts":["0","0","0","0","0","0","0","0","9","0","0","7","4","0","0","29"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":91}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":510},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":3182}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"82","sum":164,"bucketCounts":["0","82","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"21","sum":53340,"bucketCounts":["0","0","0","0","0","0","0","0","0","18","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"82","sum":1138864,"bucketCounts":["0","0","0","0","0","0","0","0","18","0","0","0","6","0","0","58"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":1}]}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"11","sum":22,"bucketCounts":["0","11","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"49","sum":98,"bucketCounts":["0","49","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601691873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601691873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601691873000000","asDouble":10000}]}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"82","sum":82,"bucketCounts":["0","82","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"50","sum":0,"bucketCounts":["50","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":0.003007746}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":103.763091826}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":1621}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601691873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601691873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601691873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":1621},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":1621}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":1.3299999999999998}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":130736408}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":3692},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":3692}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":198180864}]}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"11","sum":4202642,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"49","sum":10257884,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","49"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"11","sum":11,"bucketCounts":["0","11","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"49","sum":49,"bucketCounts":["0","49","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"11","sum":11,"bucketCounts":["0","11","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"49","sum":49,"bucketCounts":["0","49","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":323897576}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","count":"91","sum":5313,"bucketCounts":["31","3","19","26","1","1","10","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601691873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601691873000000","asDouble":1}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601691873000000","asDouble":398}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601693109832073","asInt":"1744"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601693109832073","asInt":"1744"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601693109834343","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601693109834343","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601693109836073","asInt":"3748"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601693109836073","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601693109836073","asInt":"3748"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601693109837873","asInt":"3748"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601693109837873","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601693109837873","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601693109840663","asInt":"94"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601693109841663","count":"94","sum":5492,"bucketCounts":["31","3","20","27","2","1","10","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601693109845253","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601693109847613","asInt":"1744"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601693109848443","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601693109849063","asInt":"510"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601693109849063","asInt":"3238"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601693109849773","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601693109849773","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601693109851043","asDouble":108.76361474000001}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601693109673345","asInt":"79791896"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601693109853803","asInt":"334204600"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601693109677085","asInt":"130736408"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601693109854683","asDouble":1.38}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601693109811543","asInt":"198180864"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601693109859003","count":"86","sum":451652908,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","86"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601693109859003","count":"51","sum":26649243,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","51"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":289366,"max":1121404}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601693109861643","count":"86","sum":172,"bucketCounts":["0","86","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601693109862903","count":"21","sum":53340,"bucketCounts":["0","0","0","0","0","0","0","0","0","18","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601693109862903","count":"86","sum":1245830,"bucketCounts":["0","0","0","0","0","0","0","0","18","0","0","0","6","0","0","62"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":31188}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601693109864123","count":"51","sum":0,"bucketCounts":["51","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601693109864123","count":"86","sum":86,"bucketCounts":["0","86","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601693109864943","count":"51","sum":21,"bucketCounts":["30","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601693109864943","count":"86","sum":86,"bucketCounts":["0","86","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601693109866343","count":"11","sum":4202642,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601693109866343","count":"52","sum":10699038,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","52"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601693109868173","count":"11","sum":140347,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","10"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601693109868173","count":"52","sum":621899,"bucketCounts":["0","0","0","0","0","0","0","0","9","0","0","8","4","0","0","31"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":28903}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601693109869883","count":"52","sum":104,"bucketCounts":["0","52","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601693109869883","count":"11","sum":22,"bucketCounts":["0","11","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601693109872173","count":"11","sum":11,"bucketCounts":["0","11","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601693109872173","count":"52","sum":52,"bucketCounts":["0","52","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601693109873113","count":"11","sum":11,"bucketCounts":["0","11","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601693109873113","count":"52","sum":52,"bucketCounts":["0","52","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601693109815513","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601693109827663","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601693109823253","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601693109819163","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601693109828303","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601693109824583","asInt":"10000"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601694465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601694465000000","asDouble":0.000639161}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601694465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601694465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601694465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601695439000000","asDouble":0.000544932}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601695439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601695439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601695439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601695439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":10},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":54}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":84553728}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","count":"54","sum":0.06632889699999998,"bucketCounts":["54","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601696616000000","count":"10","sum":0.026524381000000003,"bucketCounts":["10","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":86666080}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":46800}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":102230}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":20}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","count":"54","sum":0,"bucketCounts":["54","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601696616000000","count":"10","sum":74836,"bucketCounts":["0","0","10","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":418}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":19028136}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":44072960}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601691.7653491}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":10},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601695}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1474060}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":12761},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":5268972}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":54}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000"}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":33}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":9510912}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601636.42}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0.33}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":418}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":2064384}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":549920}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":65}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":12761}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":465555}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":31229328}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":685440}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":109368}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":574923}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000"}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0.003787275}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":10000}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":2064384}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":62518536}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":21749760}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":38400}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":14}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":19028136}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601696616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":54}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":418}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":8905920}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","count":"10","sum":0.001238882,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.00005985},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000213016},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601696616000000","asDouble":1}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601695439}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601637670}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1048576}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601695}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":36}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":824807424}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":0}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696616000000","asDouble":22323200}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601696616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":4066},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":344406952}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":398}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":86237040}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"90","sum":473796838,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","90"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"55","sum":28532045,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","55"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601696873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601696873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601696873000000","asDouble":0}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":1803},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":1803}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":100}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"12","sum":12,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"54","sum":54,"bucketCounts":["0","54","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601696873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601696873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601696873000000","asDouble":10000}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":198180864}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":6}]}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"23","sum":54632,"bucketCounts":["0","0","0","0","0","0","0","0","0","20","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"90","sum":1275098,"bucketCounts":["0","0","0","0","0","0","0","0","20","0","0","0","6","0","0","64"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"12","sum":4404329,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","12"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"54","sum":10977104,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","54"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"12","sum":24,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"54","sum":108,"bucketCounts":["0","54","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"12","sum":12,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"54","sum":54,"bucketCounts":["0","54","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601696873000000","asDouble":1}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":1}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":130867480}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":113.763347651}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":560},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":3506}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":1803}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"90","sum":90,"bucketCounts":["0","90","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"55","sum":23,"bucketCounts":["32","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"100","sum":5869,"bucketCounts":["34","3","21","28","2","1","11","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"12","sum":154247,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"54","sum":636463,"bucketCounts":["0","0","0","0","0","0","0","0","10","0","0","8","4","0","0","32"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":4066},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":4066}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","asDouble":1.43}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"90","sum":180,"bucketCounts":["0","90","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"90","sum":90,"bucketCounts":["0","90","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601696873000000","count":"55","sum":0,"bucketCounts":["55","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":0.003138094}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601696873000000","asDouble":398}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601698109256403","asInt":"1911"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601698109256403","asInt":"1911"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601698109257723","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601698109257723","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601698109259483","asInt":"4122"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601698109259483","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601698109259483","asInt":"4122"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601698109260203","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601698109260203","asInt":"4122"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601698109260203","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601698109262183","asInt":"103"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601698109263043","count":"103","sum":6033,"bucketCounts":["34","3","22","30","2","1","11","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601698109266173","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601698109267523","asInt":"1911"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601698109268303","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601698109269013","asInt":"560"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601698109269013","asInt":"3562"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601698109269603","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601698109269603","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601698109270613","asDouble":118.762788201}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601698109098965","asInt":"89102608"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601698109272573","asInt":"353717664"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601698109101595","asInt":"130867480"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601698109273233","asDouble":1.48}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601698109239843","asInt":"198180864"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601698109277113","count":"94","sum":496583749,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","94"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601698109277113","count":"56","sum":28928209,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","56"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":289366,"max":1121404}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601698109278343","count":"94","sum":188,"bucketCounts":["0","94","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601698109279213","count":"94","sum":1366780,"bucketCounts":["0","0","0","0","0","0","0","0","20","0","0","0","6","0","0","68"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":31188},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601698109279213","count":"23","sum":54632,"bucketCounts":["0","0","0","0","0","0","0","0","0","20","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601698109280293","count":"56","sum":0,"bucketCounts":["56","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601698109280293","count":"94","sum":94,"bucketCounts":["0","94","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601698109281033","count":"56","sum":23,"bucketCounts":["33","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601698109281033","count":"94","sum":94,"bucketCounts":["0","94","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601698109283023","count":"12","sum":4404329,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","12"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601698109283023","count":"57","sum":11504446,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","57"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601698109284883","count":"12","sum":154247,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601698109284883","count":"57","sum":682288,"bucketCounts":["0","0","0","0","0","0","0","0","10","0","0","9","4","0","0","34"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":28903}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601698109286003","count":"12","sum":24,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601698109286003","count":"57","sum":114,"bucketCounts":["0","57","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601698109286813","count":"12","sum":12,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601698109286813","count":"57","sum":57,"bucketCounts":["0","57","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601698109287643","count":"12","sum":12,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601698109287643","count":"57","sum":57,"bucketCounts":["0","57","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601698109254013","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601698109249663","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601698109243203","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601698109245953","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601698109254673","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601698109250553","asInt":"10000"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601699465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601699465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601699465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601699465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601699465000000","asDouble":0.000869437}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601700439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601700439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601700439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601700439000000","asDouble":0.00065244}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601700439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":13999},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1474444}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":38400}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":92587888}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":615546}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601636.42}]}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1252}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":418}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","count":"59","sum":0,"bucketCounts":["59","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601701616000000","count":"11","sum":82334,"bucketCounts":["0","0","11","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0.35}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":11},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":59}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":472355}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":9412608}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":685440}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601637670}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":11},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000"}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601691.7653491}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":5268588}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":1252}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":62518536}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":13999}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":418}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":59}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":24949944}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601700439}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":31229328}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":10000}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601637670}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","count":"10","sum":0.001238882,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.00005985},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000213016},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":44040192}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":621600}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":84553728}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000"}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":8905920}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":36}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":14}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":824807424}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":70}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1048576}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":2097152}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601701616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":59}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":46800}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601700}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":24949944}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":108624}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":28647424}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601700}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":418}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":20}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0.003566998}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601701616000000","asDouble":1}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":143191}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":2097152}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","count":"59","sum":0.07145821299999998,"bucketCounts":["59","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601701616000000","count":"11","sum":0.028650780000000004,"bucketCounts":["11","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1}]}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0.003792276}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":33}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":15392768}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":0}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601701616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"98","sum":196,"bucketCounts":["0","98","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"13","sum":13,"bucketCounts":["0","13","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"59","sum":59,"bucketCounts":["0","59","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":1}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":1.52}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601701873000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":398}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":398}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"98","sum":513858349,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","98"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"60","sum":31064819,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","60"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"25","sum":55924,"bucketCounts":["0","0","0","0","0","0","0","0","0","22","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"98","sum":1395872,"bucketCounts":["0","0","0","0","0","0","0","0","22","0","0","0","6","0","0","70"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"13","sum":26,"bucketCounts":["0","13","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"59","sum":118,"bucketCounts":["0","59","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":4440},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":4440}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":123.763593118}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"109","sum":6409,"bucketCounts":["37","3","23","31","2","1","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":4440},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":1969},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":1969}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"98","sum":98,"bucketCounts":["0","98","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"60","sum":0,"bucketCounts":["60","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"13","sum":168147,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","12"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"59","sum":696766,"bucketCounts":["0","0","0","0","0","0","0","0","11","0","0","9","4","0","0","35"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601701873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601701873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601701873000000","asDouble":10000}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":363092856}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":6}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":1969}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"13","sum":4565116,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"59","sum":11798462,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","59"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":130867480}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"13","sum":13,"bucketCounts":["0","13","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"59","sum":59,"bucketCounts":["0","59","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601701873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601701873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601701873000000","asDouble":0}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":198967296}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":95612232}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":109}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":610},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","asDouble":3830}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"98","sum":98,"bucketCounts":["0","98","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601701873000000","count":"60","sum":25,"bucketCounts":["35","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601701873000000","asDouble":0.003258573}]}}]}]},{"resource":{"attributes":[{"key":"process.runtime.description","value":{"stringValue":"go version go1.21.3 linux/amd64"}},{"key":"process.runtime.name","value":{"stringValue":"go"}},{"key":"process.runtime.version","value":{"stringValue":"go1.21.3"}},{"key":"service.name","value":{"stringValue":"go-faster.oteldb.chotel"}},{"key":"telemetry.sdk.language","value":{"stringValue":"go"}},{"key":"telemetry.sdk.name","value":{"stringValue":"opentelemetry"}},{"key":"telemetry.sdk.version","value":{"stringValue":"1.21.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/runtime","version":"0.46.0"},"metrics":[{"name":"runtime.uptime","description":"Milliseconds since application was initialized","unit":"ms","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917005741","timeUnixNano":"1700601701918714326","asInt":"60001"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.goroutines","description":"Number of goroutines that currently exist","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917011171","timeUnixNano":"1700601701918717926","asInt":"25"}],"aggregationTemporality":2}},{"name":"process.runtime.go.cgo.calls","description":"Number of cgo calls made by the current process","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917013591","timeUnixNano":"1700601701918718856","asInt":"0"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_alloc","description":"Bytes of allocated heap objects","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917028410","timeUnixNano":"1700601701918719656","asInt":"3792864"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_idle","description":"Bytes in idle (unused) spans","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917032650","timeUnixNano":"1700601701918720256","asInt":"9797632"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_inuse","description":"Bytes in in-use spans","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917034590","timeUnixNano":"1700601701918720726","asInt":"5505024"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_objects","description":"Number of allocated heap objects","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917036120","timeUnixNano":"1700601701918720986","asInt":"6799"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_released","description":"Bytes of idle spans whose physical memory has been returned to the OS","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917038040","timeUnixNano":"1700601701918721246","asInt":"8339456"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_sys","description":"Bytes of heap memory obtained from the OS","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917042550","timeUnixNano":"1700601701918721546","asInt":"15302656"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.lookups","description":"Number of pointer lookups performed by the runtime","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917056380","timeUnixNano":"1700601701918721936","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.mem.live_objects","description":"Number of live objects is the number of cumulative Mallocs - Frees","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917057580","timeUnixNano":"1700601701918722176","asInt":"6799"}],"aggregationTemporality":2}},{"name":"process.runtime.go.gc.count","description":"Number of completed garbage collection cycles","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917058830","timeUnixNano":"1700601701918722436","asInt":"12"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.gc.pause_total_ns","description":"Cumulative nanoseconds in GC stop-the-world pauses since the program started","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917064450","timeUnixNano":"1700601701918722695","asInt":"552767"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.gc.pause_ns","description":"Amount of nanoseconds in GC stop-the-world pauses","histogram":{"dataPoints":[{"startTimeUnixNano":"1700601641917074040","timeUnixNano":"1700601701918723205","count":"12","sum":552767,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","12"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":22100,"max":102909}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.1"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931833929","timeUnixNano":"1700601701918727375","count":"12","sum":7.926315999999999,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0.390424,"max":2.09179}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931840749","timeUnixNano":"1700601701918729045","count":"12","sum":24,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931845139","timeUnixNano":"1700601701918730155","count":"12","sum":270473,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","1","0","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":4794,"max":28903}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931849989","timeUnixNano":"1700601701918731335","count":"12","sum":12,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931852069","timeUnixNano":"1700601701918732115","count":"12","sum":12,"bucketCounts":["0","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.21.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601703109999208","count":"102","sum":532047977,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","102"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601703109999208","count":"61","sum":31599301,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","61"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":289366,"max":1121404}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601703110003147","count":"102","sum":204,"bucketCounts":["0","102","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601703110005217","count":"102","sum":1487542,"bucketCounts":["0","0","0","0","0","0","0","0","22","0","0","0","6","0","0","74"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":31188},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601703110005217","count":"25","sum":55924,"bucketCounts":["0","0","0","0","0","0","0","0","0","22","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601703110006697","count":"61","sum":0,"bucketCounts":["61","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601703110006697","count":"102","sum":102,"bucketCounts":["0","102","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601703110007747","count":"61","sum":25,"bucketCounts":["36","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601703110007747","count":"102","sum":102,"bucketCounts":["0","102","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601703110008817","count":"14","sum":4888501,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","14"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601703110008817","count":"62","sum":12255736,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","62"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601703110010537","count":"14","sum":173357,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","12"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601703110010537","count":"62","sum":742585,"bucketCounts":["0","0","0","0","0","0","0","0","11","0","0","10","4","0","0","37"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":28903}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601703110011817","count":"14","sum":28,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601703110011817","count":"62","sum":124,"bucketCounts":["0","62","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601703110012827","count":"14","sum":14,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601703110012827","count":"62","sum":62,"bucketCounts":["0","62","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601703110013587","count":"62","sum":62,"bucketCounts":["0","62","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601703110013587","count":"14","sum":14,"bucketCounts":["0","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601703109984908","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601703109996038","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601703109992248","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601703109988488","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601703109996618","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601703109993358","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601703110019037","asInt":"2077"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601703110019037","asInt":"2077"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601703110019947","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601703110019947","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601703110021357","asInt":"4515"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601703110021357","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601703110021357","asInt":"4515"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601703110022077","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601703110022077","asInt":"4515"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601703110022077","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601703110023767","asInt":"112"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601703110024597","count":"112","sum":6592,"bucketCounts":["37","3","24","33","2","1","12","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601703110026797","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601703110027777","asInt":"2077"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601703110028467","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601703110029097","asInt":"629"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601703110029097","asInt":"3886"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601703110029787","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601703110029787","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601703110031277","asDouble":128.76433246300002}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601703109853790","asInt":"99989528"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601703110032967","asInt":"373979776"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601703109861080","asInt":"130867480"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601703110033697","asDouble":1.56}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601703109980738","asInt":"200278016"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601704465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601704465000000","asDouble":0.00065276}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601704465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601704465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601704465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601705439000000","asDouble":0.00064129}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601705439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601705439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601705439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601705439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":560817}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":46800}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":76}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":64}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000"}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0.003622678}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":85602304}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0.003792276}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601706616000000","asDouble":1}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":2064384}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":15359},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":104221}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":100338480}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601636.42}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":15359}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":418}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":22192128}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":44072960}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":62518536}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1475196}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":5265788}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":824807424}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":665038}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":118929}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":30825752}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","count":"64","sum":0,"bucketCounts":["64","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601706616000000","count":"12","sum":89852,"bucketCounts":["0","0","12","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601705439}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601637670}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":18519312}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":10000}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":38400}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601706616000000","asDouble":1}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":12},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":64}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":18519312}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","count":"64","sum":0.07857885999999996,"bucketCounts":["64","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601706616000000","count":"12","sum":0.030801499000000003,"bucketCounts":["12","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601705}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":418}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":510080}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601705}]}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":21880832}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":9003008}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":64}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":8907968}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1700601702.0169818}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":14}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000"}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","count":"11","sum":0.001346451,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.00005985},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000213016},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":20}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":36}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":1048576}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":33}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":418}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":0}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":12},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":685440}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706616000000","asDouble":2064384}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0.38}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601706616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":679},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":4154}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"106","sum":106,"bucketCounts":["0","106","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"65","sum":27,"bucketCounts":["38","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"15","sum":15,"bucketCounts":["0","15","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"64","sum":64,"bucketCounts":["0","64","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601706873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601706873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601706873000000","asDouble":0}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":4833},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":4833}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":202375168}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"118","sum":6977,"bucketCounts":["40","3","25","34","2","1","13","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601706873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":0.002472404}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":384835064}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"106","sum":549928666,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","106"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"65","sum":33835638,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","65"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":1}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":2144}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"15","sum":15,"bucketCounts":["0","15","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"64","sum":64,"bucketCounts":["0","64","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":4833},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":2144},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":2144}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":133.763880561}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601706873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601706873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601706873000000","asDouble":10000}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":118}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":398}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":398}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":106467520}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":131129624}]}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"15","sum":30,"bucketCounts":["0","15","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"64","sum":128,"bucketCounts":["0","64","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"106","sum":212,"bucketCounts":["0","106","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"15","sum":5082988,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","15"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"64","sum":12537601,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","64"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"15","sum":187257,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"64","sum":758121,"bucketCounts":["0","0","0","0","0","0","0","0","11","0","1","10","4","0","0","38"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","asDouble":1.6099999999999999}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"106","sum":106,"bucketCounts":["0","106","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"65","sum":0,"bucketCounts":["65","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"27","sum":57216,"bucketCounts":["0","0","0","0","0","0","0","0","0","24","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601706873000000","count":"106","sum":1518782,"bucketCounts":["0","0","0","0","0","0","0","0","22","0","2","0","6","0","0","76"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601706873000000","asDouble":398}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601708109800918","asDouble":138.76365853200002}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601708109633831","asInt":"56875744"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601708109804928","asInt":"395950776"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601708109640301","asInt":"139780376"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601708109806248","asDouble":1.69}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601708109785589","asInt":"204210176"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601708109810988","count":"66","sum":34507748,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","66"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":289366,"max":1121404},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601708109810988","count":"110","sum":568029083,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","110"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601708109813458","count":"110","sum":220,"bucketCounts":["0","110","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601708109815088","count":"110","sum":1600146,"bucketCounts":["0","0","0","0","0","0","0","0","22","0","2","0","6","0","0","80"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":31188},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601708109815088","count":"27","sum":57216,"bucketCounts":["0","0","0","0","0","0","0","0","0","24","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601708109817158","count":"66","sum":0,"bucketCounts":["66","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601708109817158","count":"110","sum":110,"bucketCounts":["0","110","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601708109818058","count":"66","sum":27,"bucketCounts":["39","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601708109818058","count":"110","sum":110,"bucketCounts":["0","110","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601708109819018","count":"15","sum":5082988,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","15"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601708109819018","count":"67","sum":13038334,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","67"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601708109821008","count":"15","sum":187257,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601708109821008","count":"67","sum":798789,"bucketCounts":["0","0","0","0","0","0","0","0","11","0","1","11","4","0","0","40"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":28903}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601708109822288","count":"15","sum":30,"bucketCounts":["0","15","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601708109822288","count":"67","sum":134,"bucketCounts":["0","67","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601708109823298","count":"15","sum":15,"bucketCounts":["0","15","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601708109823298","count":"67","sum":67,"bucketCounts":["0","67","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601708109824218","count":"15","sum":15,"bucketCounts":["0","15","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601708109824218","count":"67","sum":67,"bucketCounts":["0","67","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601708109788478","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601708109798808","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601708109794448","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601708109790498","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601708109799358","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601708109795468","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601708109828578","asInt":"2240"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601708109828578","asInt":"2240"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601708109829058","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601708109829058","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601708109830048","asInt":"4889"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601708109830048","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601708109830048","asInt":"4889"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601708109830778","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601708109830778","asInt":"4889"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601708109830778","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601708109832978","asInt":"121"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601708109833478","count":"121","sum":7129,"bucketCounts":["40","3","26","36","2","1","13","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601708109836418","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601708109837238","asInt":"2240"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601708109837838","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601708109838418","asInt":"679"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601708109838418","asInt":"4210"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601708109839108","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601708109839108","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601709465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601709465000000","asDouble":0.000584662}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601709465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601709465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601709465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601710439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601710439000000","asDouble":0.000558902}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601710439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601710439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601710439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":20}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":23607784}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":27516928}]}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":33}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601711616000000","asDouble":1}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":81}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":105426952}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":137684}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0.4}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601710}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","count":"11","sum":0.001346451,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.00005985},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000213016},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":30825752}]}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":9003008}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":10000}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":23607784}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":38400}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":69}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":14}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601711616000000","asDouble":1}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601710439}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":16523264}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":36}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":16597},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":16597}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","count":"69","sum":0.08639113799999996,"bucketCounts":["69","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601711616000000","count":"13","sum":0.032657662000000004,"bucketCounts":["13","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":125324}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0.004936628}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":2097152}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":824807424}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601702.0169818}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":685440}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":418}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":2097152}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":69}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":13},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":418}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":13},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":69}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1475588}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":8907968}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","count":"69","sum":0,"bucketCounts":["69","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601711616000000","count":"13","sum":97365,"bucketCounts":["0","0","13","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":592000}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":62518536}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":568011}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":705695}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":44040192}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000"}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":418}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":5265396}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":85602304}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601710}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601636.42}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000"}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601711616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":46800}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711616000000","asDouble":1048576}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":2298},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":2298}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":6}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":729},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":4478}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601711873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601711873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601711873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601711873000000","asDouble":10000}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":5207},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":408793312}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":2298}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"16","sum":5233846,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","16"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"69","sum":13328130,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","69"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"16","sum":201157,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","14"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"69","sum":813267,"bucketCounts":["0","0","0","0","0","0","0","0","12","0","1","11","4","0","0","41"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"16","sum":32,"bucketCounts":["0","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"69","sum":138,"bucketCounts":["0","69","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"114","sum":228,"bucketCounts":["0","114","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":0.0028057}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":139780376}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":127}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":5207},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":5207}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":1.77}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"114","sum":587212446,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","114"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"70","sum":36647368,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","70"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":398}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":204210176}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":65080560}]}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"114","sum":114,"bucketCounts":["0","114","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"70","sum":0,"bucketCounts":["70","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"114","sum":114,"bucketCounts":["0","114","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"70","sum":29,"bucketCounts":["41","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"16","sum":16,"bucketCounts":["0","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"69","sum":69,"bucketCounts":["0","69","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":1}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601711873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601711873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601711873000000","asDouble":0}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","asDouble":143.763966294}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"127","sum":7505,"bucketCounts":["43","3","27","37","2","1","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"29","sum":58508,"bucketCounts":["0","0","0","0","0","0","0","0","0","26","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"114","sum":1629238,"bucketCounts":["0","0","0","0","0","0","0","0","24","0","2","0","6","0","0","82"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"16","sum":16,"bucketCounts":["0","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601711873000000","count":"69","sum":69,"bucketCounts":["0","69","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601711873000000","asDouble":398}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601713110134276","asInt":"2400"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601713110134276","asInt":"2400"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601713110138466","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601713110138466","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601713110140166","asInt":"5263"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601713110140166","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601713110140166","asInt":"5263"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601713110141595","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601713110141595","asInt":"5263"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601713110141595","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601713110144115","asInt":"130"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601713110145135","count":"130","sum":7663,"bucketCounts":["43","3","28","39","2","1","14","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601713110148885","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601713110150215","asInt":"2400"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601713110151015","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601713110151665","asInt":"729"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601713110151665","asInt":"4534"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601713110152725","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601713110152725","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601713110153665","asDouble":148.764329831}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601713109984048","asInt":"68027320"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601713110156715","asInt":"419944888"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601713109987078","asInt":"139780376"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601713110157575","asDouble":1.81}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601713110117496","asInt":"204472320"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601713110161325","count":"118","sum":607399944,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","118"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":597251,"max":16407354},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601713110161325","count":"71","sum":37073272,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","71"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":289366,"max":1121404}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601713110163815","count":"118","sum":236,"bucketCounts":["0","118","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601713110164475","count":"118","sum":1719472,"bucketCounts":["0","0","0","0","0","0","0","0","24","0","2","0","6","0","0","86"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":31188},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601713110164475","count":"29","sum":58508,"bucketCounts":["0","0","0","0","0","0","0","0","0","26","0","0","0","0","0","3"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601713110165615","count":"71","sum":0,"bucketCounts":["71","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601713110165615","count":"118","sum":118,"bucketCounts":["0","118","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601713110166525","count":"118","sum":118,"bucketCounts":["0","118","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601713110166525","count":"71","sum":29,"bucketCounts":["42","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601713110167605","count":"16","sum":5233846,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","16"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601713110167605","count":"72","sum":13811243,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","72"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":98059,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601713110169405","count":"16","sum":201157,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","14"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601713110169405","count":"72","sum":858368,"bucketCounts":["0","0","0","0","0","0","0","0","12","0","1","12","4","0","0","43"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":28903}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601713110171065","count":"16","sum":32,"bucketCounts":["0","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601713110171065","count":"72","sum":144,"bucketCounts":["0","72","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601713110171885","count":"16","sum":16,"bucketCounts":["0","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601713110171885","count":"72","sum":72,"bucketCounts":["0","72","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601713110172775","count":"16","sum":16,"bucketCounts":["0","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601713110172775","count":"72","sum":72,"bucketCounts":["0","72","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601713110120926","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601713110131586","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601713110127456","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601713110123786","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601713110132226","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601713110128446","asInt":"10000"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601714465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601714465000000","asDouble":0.00071739}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601714465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601714465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601714465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601715439000000","asDouble":0.000627201}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601715439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601715439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601715439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601715439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":418}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":2064384}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":418}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":36}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":418}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":86126592}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601715}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601715}]}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":14},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":112596776}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":14},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":74}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":17672464}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":17835}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1360}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":100400}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":508000}]}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000"}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":22904832}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","count":"74","sum":0.09314787099999995,"bucketCounts":["74","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601716616000000","count":"14","sum":0.035933254000000005,"bucketCounts":["14","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":21168128}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601716616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":5776292}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":131674}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":20}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1048576}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":825331712}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":46800}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0.43}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000"}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":752056}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601716616000000","asDouble":1}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":651656}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601636.42}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":10000}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":33}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":4}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":14}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":8920280}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601712.0676582}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1476668}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":2064384}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601715439}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601637}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","count":"74","sum":0,"bucketCounts":["74","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601716616000000","count":"14","sum":104883,"bucketCounts":["0","0","14","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":7798784}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":31939064}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":17835},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":74}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","count":"12","sum":0.00140531,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.00005985},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000213016},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":74}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":38400}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":685440}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0.003576039}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":17672464}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":87}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":0}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":44072960}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":63042824}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601716616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":2459},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":2459}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":74624496}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":6}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601716873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601716873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601716873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":5581},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"122","sum":626042745,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","122"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"75","sum":39016103,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","75"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"122","sum":244,"bucketCounts":["0","122","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"32","sum":73700,"bucketCounts":["0","0","0","0","0","0","0","0","0","28","0","0","0","0","0","4"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"122","sum":1748740,"bucketCounts":["0","0","0","0","0","0","0","0","26","0","2","0","6","0","0","88"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"17","sum":5386764,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","17"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"74","sum":14031709,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","74"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":204472320}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":136}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":153.764298891}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":779},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":4802}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"122","sum":122,"bucketCounts":["0","122","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"75","sum":0,"bucketCounts":["75","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601716873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601716873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601716873000000","asDouble":10000}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":139780376}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":429488824}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601716873000000","asDouble":1}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"136","sum":8040,"bucketCounts":["46","3","29","40","2","1","15","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":0.002843559}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":5581},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":5581}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":1.85}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"17","sum":215057,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","15"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"74","sum":872932,"bucketCounts":["0","0","0","0","0","0","0","0","13","0","1","12","4","0","0","44"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"17","sum":34,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"74","sum":148,"bucketCounts":["0","74","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":398}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601716873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"122","sum":122,"bucketCounts":["0","122","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"75","sum":32,"bucketCounts":["43","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"17","sum":17,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"74","sum":74,"bucketCounts":["0","74","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","asDouble":2459}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"17","sum":17,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601716873000000","count":"74","sum":74,"bucketCounts":["0","74","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601718109730708","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601718109742018","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601718109737538","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601718109733778","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601718109742578","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601718109738808","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601718109748128","asInt":"2591"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601718109748128","asInt":"2591"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601718109749318","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601718109749318","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601718109750168","asInt":"5637"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601718109750168","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601718109750168","asInt":"5637"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601718109750868","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601718109750868","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601718109750868","asInt":"5637"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601718109752728","asInt":"139"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601718109753818","count":"139","sum":8228,"bucketCounts":["46","3","30","41","3","1","15","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601718109757058","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601718109758098","asInt":"2591"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601718109758638","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601718109759328","asInt":"779"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601718109759328","asInt":"4858"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601718109759838","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601718109759838","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601718109760988","asDouble":158.76391695299998}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601718109597620","asInt":"77634064"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601718109762538","asInt":"439095568"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601718109600490","asInt":"139780376"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601718109763388","asDouble":1.9}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601718109727308","asInt":"204734464"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601718109766478","count":"76","sum":39294509,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","76"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":278406,"max":1121404},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601718109766478","count":"126","sum":647873507,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","126"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572682,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601718109767488","count":"126","sum":252,"bucketCounts":["0","126","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601718109768118","count":"33","sum":87654,"bucketCounts":["0","0","0","0","0","0","0","0","0","28","0","0","0","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601718109768118","count":"126","sum":1860540,"bucketCounts":["0","0","0","0","0","0","0","0","26","0","2","0","6","0","0","92"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601718109769688","count":"76","sum":0,"bucketCounts":["76","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601718109769688","count":"126","sum":126,"bucketCounts":["0","126","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601718109770308","count":"76","sum":33,"bucketCounts":["43","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601718109770308","count":"126","sum":126,"bucketCounts":["0","126","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601718109771148","count":"77","sum":14527192,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","77"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":93018,"max":1750565},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601718109771148","count":"17","sum":5386764,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","17"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":149648,"max":2029081}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601718109772598","count":"17","sum":215057,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","15"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601718109772598","count":"77","sum":928814,"bucketCounts":["0","0","0","0","0","0","0","0","13","0","1","12","5","0","0","46"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601718109773798","count":"17","sum":34,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601718109773798","count":"77","sum":154,"bucketCounts":["0","77","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601718109774678","count":"17","sum":17,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601718109774678","count":"77","sum":77,"bucketCounts":["0","77","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601718109775468","count":"17","sum":17,"bucketCounts":["0","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601718109775468","count":"77","sum":77,"bucketCounts":["0","77","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601719465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601719465000000","asDouble":0.000596551}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601719465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601719465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601719465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601720439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601720439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601720439000000","asDouble":0.000517723}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601720439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601720439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1477068}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":19073},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":138101}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":658302}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":2097152}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":92}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":15},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":79}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":418}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":63042824}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":27238400}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0.45}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","count":"79","sum":0,"bucketCounts":["79","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601721616000000","count":"15","sum":112392,"bucketCounts":["0","0","15","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":791377}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601637}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601720}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","count":"12","sum":0.00140531,"quantileValues":[{"value":0.00004626},{"quantile":0.25,"value":0.00005985},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000213016},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":23532784}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":133075}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":2097152}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1048576}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","count":"79","sum":0.09828380599999996,"bucketCounts":["79","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601721616000000","count":"15","sum":0.038091213000000006,"bucketCounts":["15","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":36}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":8924400}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":33}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":16801792}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":38400}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":4}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000"}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":418}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601712.0676582}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":579680}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":685440}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":79}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":118457096}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":44040192}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601636.42}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":825331712}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":46800}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":14}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":10000}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":19073}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":86126592}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":418}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601721616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000"}]}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0.003002587}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":5771772}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601720439}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601721616000000","asDouble":1}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":7798784}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":20}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":79}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":15},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":23532784}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":31939064}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601720}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721616000000","asDouble":0}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601721616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"18","sum":18,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"79","sum":79,"bucketCounts":["0","79","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"18","sum":18,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"79","sum":79,"bucketCounts":["0","79","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601721873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":0.00267374}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":163.764110716}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":829},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":5126}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"130","sum":130,"bucketCounts":["0","130","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"80","sum":0,"bucketCounts":["80","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":1}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"145","sum":8604,"bucketCounts":["49","3","31","42","3","1","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"130","sum":663767066,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","130"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"80","sum":41325060,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","80"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"130","sum":260,"bucketCounts":["0","130","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601721873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601721873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601721873000000","asDouble":10000}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":1.95}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":84211728}]}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"18","sum":36,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"79","sum":158,"bucketCounts":["0","79","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":398}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":204996608}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":6}]}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"130","sum":130,"bucketCounts":["0","130","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"80","sum":35,"bucketCounts":["45","35","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"35","sum":88946,"bucketCounts":["0","0","0","0","0","0","0","0","0","30","0","0","0","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"130","sum":1889632,"bucketCounts":["0","0","0","0","0","0","0","0","28","0","2","0","6","0","0","94"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601721873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601721873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601721873000000","asDouble":0}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":145}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":2649}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":5955},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":5955}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"18","sum":228957,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","16"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"79","sum":943292,"bucketCounts":["0","0","0","0","0","0","0","0","14","0","1","12","5","0","0","47"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":398}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":5955},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":448682800}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":2649},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","asDouble":2649}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601721873000000","asDouble":139845912}]}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"18","sum":5527332,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","18"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601721873000000","count":"79","sum":14824237,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","79"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601723109636312","asInt":"2767"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601723109636312","asInt":"2767"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601723109637932","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601723109637932","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601723109639552","asInt":"6011"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601723109639552","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601723109639552","asInt":"6011"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601723109640502","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601723109640502","asInt":"6011"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601723109640502","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601723109642362","asInt":"148"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601723109643272","count":"148","sum":8778,"bucketCounts":["49","3","32","43","4","1","16","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601723109646892","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601723109648162","asInt":"2767"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601723109648872","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601723109649452","asInt":"829"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601723109649452","asInt":"5182"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601723109650132","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601723109650132","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601723109651202","asDouble":168.763969831}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601723109471894","asInt":"87155680"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601723109653272","asInt":"458204416"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601723109476914","asInt":"139845912"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601723109654022","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601723109616682","asInt":"204996608"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601723109657472","count":"81","sum":41634185,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","81"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":278406,"max":1121404},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601723109657472","count":"134","sum":686137691,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","134"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572682,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601723109659182","count":"134","sum":268,"bucketCounts":["0","134","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601723109660222","count":"134","sum":1989156,"bucketCounts":["0","0","0","0","0","0","0","0","28","0","2","0","6","0","0","98"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601723109660222","count":"35","sum":88946,"bucketCounts":["0","0","0","0","0","0","0","0","0","30","0","0","0","0","0","5"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601723109661712","count":"81","sum":0,"bucketCounts":["81","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601723109661712","count":"134","sum":134,"bucketCounts":["0","134","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601723109662572","count":"81","sum":35,"bucketCounts":["46","35","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601723109662572","count":"134","sum":134,"bucketCounts":["0","134","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601723109663472","count":"18","sum":5527332,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","18"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":140568,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601723109663472","count":"82","sum":15299169,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","82"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":93018,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601723109664872","count":"18","sum":228957,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","16"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601723109664872","count":"82","sum":993038,"bucketCounts":["0","0","0","0","0","0","0","0","14","0","1","13","5","0","0","49"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601723109666052","count":"18","sum":36,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601723109666052","count":"82","sum":164,"bucketCounts":["0","82","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601723109667812","count":"18","sum":18,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601723109667812","count":"82","sum":82,"bucketCounts":["0","82","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601723109668732","count":"18","sum":18,"bucketCounts":["0","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601723109668732","count":"82","sum":82,"bucketCounts":["0","82","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601723109620792","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601723109633212","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601723109628842","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601723109624342","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601723109633842","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601723109630222","asInt":"10000"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601724465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601724465000000","asDouble":0.000753249}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601724465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601724465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601724465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601725439000000","asDouble":0.000616311}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601725439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601725439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601725439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601725439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":84}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":36}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":10000}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":24526848}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":16169584}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601637}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":84996096}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1360}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0.002930427}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601723.118798}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601726616000000","asDouble":1}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":486400}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":418}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1477844}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0.48}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000"}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601637670}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":33}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":44072960}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601725}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":14}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601636.42}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":46800}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","count":"84","sum":0,"bucketCounts":["84","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601726616000000","count":"16","sum":119901,"bucketCounts":["0","0","16","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":5760708}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":2064384}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":20311},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":9175040}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":685440}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":825331712}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":16},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":125499056}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":16169584}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":418}]}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000"}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":8934688}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":837483}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","count":"84","sum":0.10416276999999997,"bucketCounts":["84","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601726616000000","count":"16","sum":0.039757569000000006,"bucketCounts":["16","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":16},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":84}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":4}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601726616000000","asDouble":1}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":20}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":144470}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","count":"13","sum":0.00144833,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000059438},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":63042824}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601725439}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":31010696}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":84}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":97}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":19546112}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":90419}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601725}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1048576}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":38400}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":20311}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":1700601637670}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":418}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":747064}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726616000000","asDouble":2064384}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601726616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601726873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601726873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601726873000000","asDouble":0}]}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"19","sum":242857,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","17"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"84","sum":1007602,"bucketCounts":["0","0","0","0","0","0","0","0","15","0","1","13","5","0","0","50"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":6329},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":6329}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":2826},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":2826}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":2.04}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":467579680}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":0.002432295}]}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601726873000000","asDouble":1}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":1}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":879},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":5450}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"38","sum":104138,"bucketCounts":["0","0","0","0","0","0","0","0","0","32","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"138","sum":2018424,"bucketCounts":["0","0","0","0","0","0","0","0","30","0","2","0","6","0","0","100"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"19","sum":19,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"84","sum":84,"bucketCounts":["0","84","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":93586992}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":173.763643098}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"154","sum":9155,"bucketCounts":["52","3","33","44","4","1","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"138","sum":138,"bucketCounts":["0","138","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"85","sum":0,"bucketCounts":["85","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"19","sum":38,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"84","sum":168,"bucketCounts":["0","84","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601726873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601726873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601726873000000","asDouble":10000}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":6329},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"19","sum":19,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"84","sum":84,"bucketCounts":["0","84","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":6}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":154}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","asDouble":2826}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"138","sum":276,"bucketCounts":["0","138","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":398}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":204996608}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":139845912}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"138","sum":703502156,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","138"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"85","sum":44423206,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","85"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"138","sum":138,"bucketCounts":["0","138","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"85","sum":38,"bucketCounts":["47","38","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"19","sum":5689649,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","19"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601726873000000","count":"84","sum":15568865,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","84"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601726873000000","asDouble":398}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601728109790849","asDouble":178.763807764}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601728109626512","asInt":"96442232"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601728109795089","asInt":"476866232"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601728109629412","asInt":"139845912"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601728109796489","asDouble":2.0700000000000003}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601728109770029","asInt":"204996608"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601728109801839","count":"86","sum":44793871,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","86"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":278406,"max":1121934},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601728109801839","count":"142","sum":722632187,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","142"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572682,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601728109804459","count":"142","sum":284,"bucketCounts":["0","142","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601728109806189","count":"142","sum":2105502,"bucketCounts":["0","0","0","0","0","0","0","0","30","0","2","0","6","0","0","104"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601728109806189","count":"38","sum":104138,"bucketCounts":["0","0","0","0","0","0","0","0","0","32","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601728109807559","count":"86","sum":0,"bucketCounts":["86","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601728109807559","count":"142","sum":142,"bucketCounts":["0","142","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601728109808539","count":"86","sum":38,"bucketCounts":["48","38","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601728109808539","count":"142","sum":142,"bucketCounts":["0","142","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601728109810189","count":"19","sum":5689649,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","19"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":140568,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601728109810189","count":"87","sum":15932730,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","87"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":84998,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601728109812079","count":"87","sum":1051125,"bucketCounts":["0","0","0","0","0","0","0","0","15","0","1","14","5","0","0","52"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601728109812079","count":"19","sum":242857,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","17"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601728109813609","count":"19","sum":38,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601728109813609","count":"87","sum":174,"bucketCounts":["0","87","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601728109814919","count":"19","sum":19,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601728109814919","count":"87","sum":87,"bucketCounts":["0","87","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601728109816679","count":"19","sum":19,"bucketCounts":["0","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601728109816679","count":"87","sum":87,"bucketCounts":["0","87","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601728109774589","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601728109787329","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601728109783049","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601728109778689","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601728109787879","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601728109784289","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601728109821489","asInt":"2926"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601728109821489","asInt":"2926"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601728109822229","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601728109822229","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601728109823789","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601728109823789","asInt":"6385"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601728109823789","asInt":"6385"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601728109824709","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601728109824709","asInt":"6385"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601728109824709","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601728109827179","asInt":"157"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601728109827789","count":"157","sum":9311,"bucketCounts":["52","3","34","46","4","1","17","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601728109830159","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601728109831339","asInt":"2926"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601728109832229","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601728109832839","asInt":"879"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601728109832839","asInt":"5506"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601728109833459","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601728109833459","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601729465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601729465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601729465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601729465000000","asDouble":0.000766039}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601729465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601730439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601730439000000","asDouble":0.00069247}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601730439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601730439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601730439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":20}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":25550848}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":9084928}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":418}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":131264472}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":123762}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":21549}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601730}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":558080}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601731616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":877841}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","count":"89","sum":0,"bucketCounts":["89","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601731616000000","count":"17","sum":127420,"bucketCounts":["0","0","17","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":418}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":63042824}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":10000}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","count":"89","sum":0.10885409299999997,"bucketCounts":["89","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601731616000000","count":"17","sum":0.04140508500000001,"bucketCounts":["17","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601730}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601637}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":21935000}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":5760324}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000"}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":4}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","count":"13","sum":0.00144833,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000059438},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":17},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":2097152}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601731616000000","asDouble":1}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":31010696}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":2097152}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601637670}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":8934688}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":21935000}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000"}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601730439}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":33}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":21549},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0.49}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":17},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":89}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":150901}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":18489344}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":38400}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":36}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":14}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":84996096}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":44040192}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601636.42}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":825331712}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1478228}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":89}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601637}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":754079}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1700601723.118798}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":685440}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":89}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":102}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0.004213978}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":418}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":46800}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":1048576}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601731616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731616000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":163}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"146","sum":292,"bucketCounts":["0","146","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":2984},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":2984}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":102947536}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"163","sum":9687,"bucketCounts":["55","3","35","47","4","1","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":6}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"146","sum":146,"bucketCounts":["0","146","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"90","sum":0,"bucketCounts":["90","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"20","sum":20,"bucketCounts":["0","20","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"89","sum":89,"bucketCounts":["0","89","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":1}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":2.12}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":398}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":0.003039856}]}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"20","sum":256757,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","18"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"89","sum":1065603,"bucketCounts":["0","0","0","0","0","0","0","0","16","0","1","14","5","0","0","53"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":486226776}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":6703},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"146","sum":146,"bucketCounts":["0","146","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"90","sum":40,"bucketCounts":["50","40","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":183.763754325}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":929},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":5774}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":2984}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601731873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601731873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601731873000000","asDouble":0}]}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":139845912}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"146","sum":743798689,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","146"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"90","sum":46733143,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","90"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"40","sum":105430,"bucketCounts":["0","0","0","0","0","0","0","0","0","34","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"146","sum":2134594,"bucketCounts":["0","0","0","0","0","0","0","0","32","0","2","0","6","0","0","106"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"20","sum":40,"bucketCounts":["0","20","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"89","sum":178,"bucketCounts":["0","89","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601731873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601731873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601731873000000","asDouble":10000}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":204996608}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"20","sum":5839937,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","20"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"89","sum":16163278,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","89"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"20","sum":20,"bucketCounts":["0","20","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","count":"89","sum":89,"bucketCounts":["0","89","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601731873000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":398}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601731873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":6703},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601731873000000","asDouble":6703}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601733110127971","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601733110140211","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601733110136031","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601733110131391","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601733110141341","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601733110137071","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601733110148681","asInt":"3068"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601733110148681","asInt":"3068"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601733110150211","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601733110150211","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601733110151511","asInt":"6759"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601733110151511","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601733110151511","asInt":"6759"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601733110152401","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601733110152401","asInt":"6759"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601733110152401","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601733110154841","asInt":"166"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601733110155681","count":"166","sum":9827,"bucketCounts":["55","3","36","49","4","1","18","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601733110159341","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601733110160411","asInt":"3068"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601733110161051","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601733110161621","asInt":"929"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601733110161621","asInt":"5830"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601733110162591","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601733110162591","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601733110163581","asDouble":188.76410694700002}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601733109985283","asInt":"105778400"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601733110165361","asInt":"495562944"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601733109988453","asInt":"139911448"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601733110166041","asDouble":2.17}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601733110123731","asInt":"204996608"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601733110170211","count":"91","sum":47235156,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","91"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":278406,"max":1121934},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601733110170211","count":"150","sum":762668663,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","150"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572682,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601733110172191","count":"150","sum":300,"bucketCounts":["0","150","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601733110173111","count":"150","sum":2207610,"bucketCounts":["0","0","0","0","0","0","0","0","32","0","2","0","6","0","0","110"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601733110173111","count":"40","sum":105430,"bucketCounts":["0","0","0","0","0","0","0","0","0","34","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601733110174531","count":"91","sum":0,"bucketCounts":["91","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601733110174531","count":"150","sum":150,"bucketCounts":["0","150","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601733110175481","count":"150","sum":150,"bucketCounts":["0","150","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601733110175481","count":"91","sum":40,"bucketCounts":["51","40","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601733110176571","count":"20","sum":5839937,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","20"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":140568,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601733110176571","count":"92","sum":16699740,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","92"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":84998,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601733110179000","count":"20","sum":256757,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","18"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601733110179000","count":"92","sum":1102097,"bucketCounts":["0","0","0","0","0","0","0","0","16","0","1","15","5","0","0","55"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601733110180730","count":"20","sum":40,"bucketCounts":["0","20","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601733110180730","count":"92","sum":184,"bucketCounts":["0","92","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601733110181850","count":"20","sum":20,"bucketCounts":["0","20","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601733110181850","count":"92","sum":92,"bucketCounts":["0","92","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601733110182820","count":"20","sum":20,"bucketCounts":["0","20","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601733110182820","count":"92","sum":92,"bucketCounts":["0","92","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601734465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601734465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601734465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601734465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601734465000000","asDouble":0.000595851}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601735439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601735439000000","asDouble":0.000602662}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601735439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601735439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601735439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":5747988}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601735}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":22787},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":107}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0.003477899}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":19267584}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":88481}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":138280536}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":835407}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":157329}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":2064384}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601637670}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":85254144}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601636.42}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000"}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":31104688}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":36}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601736616000000","asDouble":1}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":22787}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1360}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":38400}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0.53}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":10000}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":94}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":15388160}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":63042824}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601736616000000","asDouble":1}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":418}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":923888}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":825331712}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":8912896}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":18},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":94}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","count":"94","sum":0,"bucketCounts":["94","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601736616000000","count":"18","sum":134942,"bucketCounts":["0","0","18","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","count":"14","sum":0.001505159,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1048576}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":14}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","count":"94","sum":0.11578263099999998,"bucketCounts":["94","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601736616000000","count":"18","sum":0.043920029000000006,"bucketCounts":["18","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":94}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1478228}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":2064384}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601735}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":46800}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":8947024}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":15388160}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":44072960}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":24805376}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601733.1733809}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":499520}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":418}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":685440}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":4}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000"}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":18},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":418}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":1700601735439}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":33}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736616000000","asDouble":20}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601736616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"21","sum":21,"bucketCounts":["0","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"94","sum":94,"bucketCounts":["0","94","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"42","sum":106722,"bucketCounts":["0","0","0","0","0","0","0","0","0","36","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"154","sum":2236878,"bucketCounts":["0","0","0","0","0","0","0","0","34","0","2","0","6","0","0","112"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"154","sum":154,"bucketCounts":["0","154","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"95","sum":42,"bucketCounts":["53","42","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"21","sum":6013725,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","21"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"94","sum":16917797,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","94"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":1}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":205258752}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":3127}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"154","sum":781855943,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","154"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"95","sum":49557393,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","95"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":0.002408495}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":398}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":979},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":6098}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"21","sum":42,"bucketCounts":["0","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"94","sum":188,"bucketCounts":["0","94","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"21","sum":21,"bucketCounts":["0","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"94","sum":94,"bucketCounts":["0","94","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601736873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601736873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601736873000000","asDouble":0}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":3127},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":3127}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":140173592}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":6}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601736873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601736873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601736873000000","asDouble":10000}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":7077},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":193.763743491}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"154","sum":154,"bucketCounts":["0","154","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"95","sum":0,"bucketCounts":["95","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":172}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"154","sum":308,"bucketCounts":["0","154","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601736873000000","asDouble":1}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":507442688}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"21","sum":270657,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","19"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"94","sum":1116661,"bucketCounts":["0","0","0","0","0","0","0","0","17","0","1","15","5","0","0","56"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":7077},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":7077}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","asDouble":2.26}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601736873000000","asDouble":62605360}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601736873000000","count":"172","sum":10204,"bucketCounts":["58","3","37","50","4","1","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601738110019309","count":"96","sum":49879978,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","96"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":278406,"max":1121934},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601738110019309","count":"158","sum":803961641,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","158"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601738110023939","count":"158","sum":316,"bucketCounts":["0","158","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601738110026649","count":"42","sum":106722,"bucketCounts":["0","0","0","0","0","0","0","0","0","36","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601738110026649","count":"158","sum":2320178,"bucketCounts":["0","0","0","0","0","0","0","0","34","0","2","0","6","0","0","116"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601738110028789","count":"158","sum":158,"bucketCounts":["0","158","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601738110028789","count":"96","sum":0,"bucketCounts":["96","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601738110030449","count":"96","sum":42,"bucketCounts":["54","42","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601738110030449","count":"158","sum":158,"bucketCounts":["0","158","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601738110032009","count":"21","sum":6013725,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","21"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":140568,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601738110032009","count":"97","sum":17399930,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","97"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":84998,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601738110034129","count":"21","sum":270657,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","19"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601738110034129","count":"97","sum":1158297,"bucketCounts":["0","0","0","0","0","0","0","0","17","0","1","16","5","0","0","58"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601738110038078","count":"21","sum":42,"bucketCounts":["0","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601738110038078","count":"97","sum":194,"bucketCounts":["0","97","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601738110046608","count":"21","sum":21,"bucketCounts":["0","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601738110046608","count":"97","sum":97,"bucketCounts":["0","97","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601738110047818","count":"21","sum":21,"bucketCounts":["0","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601738110047818","count":"97","sum":97,"bucketCounts":["0","97","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601738110011709","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601738110003899","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601738110016109","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601738110007699","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601738110016659","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601738110013019","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601738110055108","asInt":"3226"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601738110055108","asInt":"3226"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601738110056198","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601738110056198","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601738110057428","asInt":"7133"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601738110057428","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601738110057428","asInt":"7133"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601738110058088","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601738110058088","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601738110058088","asInt":"7133"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601738110060118","asInt":"175"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601738110060788","count":"175","sum":10359,"bucketCounts":["58","3","38","52","4","1","19","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601738110063288","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601738110064398","asInt":"3226"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601738110065218","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601738110065798","asInt":"979"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601738110065798","asInt":"6154"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601738110066528","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601738110066528","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601738110067808","asDouble":198.763598109}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601738109827782","asInt":"66319392"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601738110069318","asInt":"520205600"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601738109830972","asInt":"140173592"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601738110070018","asDouble":2.35}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601738109999949","asInt":"205258752"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601739465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601739465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601739465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601739465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601739465000000","asDouble":0.00071933}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601740439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601740439000000","asDouble":0.000548762}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601740439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601740439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601740439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":5746804}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":4}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","count":"14","sum":0.001505159,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":2097152}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":2097152}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","count":"99","sum":0.12070322899999998,"bucketCounts":["99","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601741616000000","count":"19","sum":0.04572111300000001,"bucketCounts":["19","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":21037688}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":560960}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601740}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":112}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1479412}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601733.1733809}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":8912896}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":685440}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":36}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0.55}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":24025},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":33}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":143930064}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":10000}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":418}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":31104688}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":19},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":99}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":44040192}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000"}]}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":14}]}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":99}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":20}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":8947024}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":63042824}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601636.42}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":24025}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":19},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":21037688}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":46800}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":18923520}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":825331712}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601741616000000","asDouble":1}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000"}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":418}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":842172}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":163757}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":38400}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601740439}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1048576}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","count":"99","sum":0,"bucketCounts":["99","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601741616000000","count":"19","sum":142465,"bucketCounts":["0","0","19","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":120329}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0.004179679}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":418}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":25116672}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":85254144}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":99}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601741616000000","asDouble":1}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":962501}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":1700601740}]}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601741616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":140173592}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":203.763660755}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":1029},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":6422}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"162","sum":824477352,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","162"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"100","sum":51959657,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","100"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":1}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":72851752}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":6}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"22","sum":284557,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","20"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"99","sum":1172775,"bucketCounts":["0","0","0","0","0","0","0","0","18","0","1","16","5","0","0","59"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"22","sum":44,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"99","sum":198,"bucketCounts":["0","99","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":7451},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":7451},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":7451}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":3284},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":3284}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"181","sum":10735,"bucketCounts":["61","3","39","53","4","1","20","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"162","sum":162,"bucketCounts":["0","162","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"100","sum":44,"bucketCounts":["56","44","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"22","sum":6160353,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"99","sum":17707737,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","99"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601741873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601741873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601741873000000","asDouble":10000}]}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"162","sum":324,"bucketCounts":["0","162","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601741873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":0.00277499}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":398}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":2.39}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":530451992}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601741873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601741873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601741873000000","asDouble":0}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":181}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"22","sum":22,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"99","sum":99,"bucketCounts":["0","99","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":398}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601741873000000","asDouble":205258752}]}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"162","sum":162,"bucketCounts":["0","162","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"100","sum":0,"bucketCounts":["100","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"44","sum":108014,"bucketCounts":["0","0","0","0","0","0","0","0","0","38","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"162","sum":2349270,"bucketCounts":["0","0","0","0","0","0","0","0","36","0","2","0","6","0","0","118"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","asDouble":3284}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"22","sum":22,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601741873000000","count":"99","sum":99,"bucketCounts":["0","99","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601743109387493","asInt":"3393"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601743109387493","asInt":"3393"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601743109390083","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601743109390083","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601743109391782","asInt":"7507"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601743109391782","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601743109391782","asInt":"7507"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601743109393292","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601743109393292","asInt":"7507"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601743109393292","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601743109395532","asInt":"184"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601743109396652","count":"184","sum":10900,"bucketCounts":["61","3","40","55","4","1","20","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601743109400282","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601743109401472","asInt":"3393"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601743109402562","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601743109403232","asInt":"1029"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601743109403232","asInt":"6478"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601743109403902","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601743109403902","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601743109405112","asDouble":208.763034739}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601743109223445","asInt":"76542320"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601743109408012","asInt":"540674920"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601743109227995","asInt":"140173592"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601743109408822","asDouble":2.44}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601743109368273","asInt":"205258752"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601743109413192","count":"101","sum":52218803,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","101"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":1121934},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601743109413192","count":"166","sum":842511448,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","166"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601743109415562","count":"166","sum":332,"bucketCounts":["0","166","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601743109416762","count":"166","sum":2443790,"bucketCounts":["0","0","0","0","0","0","0","0","36","0","2","0","6","0","0","122"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601743109416762","count":"44","sum":108014,"bucketCounts":["0","0","0","0","0","0","0","0","0","38","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601743109418412","count":"101","sum":0,"bucketCounts":["101","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601743109418412","count":"166","sum":166,"bucketCounts":["0","166","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601743109419362","count":"101","sum":44,"bucketCounts":["57","44","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601743109419362","count":"166","sum":166,"bucketCounts":["0","166","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601743109420552","count":"22","sum":6160353,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":140568,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601743109420552","count":"102","sum":18494587,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","102"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":84998,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601743109422312","count":"22","sum":284557,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","20"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601743109422312","count":"102","sum":1220019,"bucketCounts":["0","0","0","0","0","0","0","0","18","0","1","16","6","0","0","61"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601743109423702","count":"22","sum":44,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601743109423702","count":"102","sum":204,"bucketCounts":["0","102","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601743109424662","count":"22","sum":22,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601743109424662","count":"102","sum":102,"bucketCounts":["0","102","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601743109425852","count":"22","sum":22,"bucketCounts":["0","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601743109425852","count":"102","sum":102,"bucketCounts":["0","102","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601743109380073","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601743109372423","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601743109384633","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601743109376053","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601743109385273","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601743109381523","asInt":"10000"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601744465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601744465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601744465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601744465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601744465000000","asDouble":0.000620051}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601745439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601745439000000","asDouble":0.000514023}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601745439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601745439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601745439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0.003665256}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":25263}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":118}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":685440}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":85254144}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":8912896}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0.56}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":642880}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601746616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":5746804}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601745}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":1004683}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601745}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1479412}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","count":"104","sum":0,"bucketCounts":["104","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601746616000000","count":"20","sum":149983,"bucketCounts":["0","0","20","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":104}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","count":"14","sum":0.001505159,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":20},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":104}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601733.1733809}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","count":"104","sum":0.12504136399999996,"bucketCounts":["104","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601746616000000","count":"20","sum":0.04795453000000001,"bucketCounts":["20","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601745439}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":170132}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":30384128}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":418}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":26284576}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1048576}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601746616000000","asDouble":1}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":20}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":825331712}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601637}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":20},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":418}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":2097152}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000"}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":104}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":14}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":26284576}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":155493}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":63042824}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":25263},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":8947024}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":2097152}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":10000}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":5}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000"}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":149176952}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":849190}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":44040192}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":38400}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":46800}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":13656064}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601636.42}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601746616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":418}]}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":1700601637670}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601746616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":0}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":33}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":31104688}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746616000000","asDouble":36}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":190}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"170","sum":861040747,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","170"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"105","sum":54555900,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","105"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"46","sum":109306,"bucketCounts":["0","0","0","0","0","0","0","0","0","40","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"170","sum":2473058,"bucketCounts":["0","0","0","0","0","0","0","0","38","0","2","0","6","0","0","124"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":3452},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":3452}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"23","sum":6357310,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","23"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"104","sum":18745553,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","104"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":0.002890688}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601746873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601746873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601746873000000","asDouble":0}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":3452}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":1079},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":6746}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"170","sum":170,"bucketCounts":["0","170","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"105","sum":0,"bucketCounts":["105","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"23","sum":298457,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","21"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"104","sum":1234583,"bucketCounts":["0","0","0","0","0","0","0","0","19","0","1","16","6","0","0","62"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":7825},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":7825}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":213.763139965}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601746873000000","asDouble":1}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":83058592}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":2.49}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"23","sum":46,"bucketCounts":["0","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"104","sum":208,"bucketCounts":["0","104","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":205258752}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"170","sum":340,"bucketCounts":["0","170","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"23","sum":23,"bucketCounts":["0","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"104","sum":104,"bucketCounts":["0","104","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601746873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601746873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601746873000000","asDouble":10000}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":7825},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"23","sum":23,"bucketCounts":["0","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"104","sum":104,"bucketCounts":["0","104","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":140173592}]}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"170","sum":170,"bucketCounts":["0","170","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"105","sum":46,"bucketCounts":["59","46","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601746873000000","asDouble":398}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","asDouble":550881760}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601746873000000","count":"190","sum":11277,"bucketCounts":["64","3","41","56","4","1","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601748109608295","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601748109620204","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601748109615985","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601748109611915","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601748109620924","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601748109617054","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601748109626724","asInt":"3559"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601748109626724","asInt":"3559"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601748109627974","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601748109627974","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601748109629584","asInt":"7881"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601748109629584","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601748109629584","asInt":"7881"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601748109630394","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601748109630394","asInt":"7881"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601748109630394","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601748109632214","asInt":"193"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601748109633024","count":"193","sum":11440,"bucketCounts":["64","3","42","58","4","1","21","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601748109636744","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601748109637814","asInt":"3559"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601748109638644","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601748109639244","asInt":"1079"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601748109639244","asInt":"6802"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601748109639894","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601748109639894","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601748109640744","asDouble":218.76340595599999}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601748109467277","asInt":"85931968"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601748109641684","asInt":"560271408"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601748109470967","asInt":"140173592"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601748109642664","asDouble":2.5300000000000002}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601748109604385","asInt":"205258752"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601748109646364","count":"174","sum":880747819,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","174"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":16407354},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601748109646364","count":"106","sum":54939114,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","106"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":1121934}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601748109648004","count":"174","sum":348,"bucketCounts":["0","174","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601748109648864","count":"46","sum":109306,"bucketCounts":["0","0","0","0","0","0","0","0","0","40","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601748109648864","count":"174","sum":2563260,"bucketCounts":["0","0","0","0","0","0","0","0","38","0","2","0","6","0","0","128"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601748109650384","count":"106","sum":0,"bucketCounts":["106","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601748109650384","count":"174","sum":174,"bucketCounts":["0","174","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601748109651584","count":"106","sum":46,"bucketCounts":["60","46","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601748109651584","count":"174","sum":174,"bucketCounts":["0","174","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601748109652824","count":"23","sum":6357310,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","23"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":140568,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601748109652824","count":"107","sum":19171938,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","107"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":84998,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601748109654324","count":"23","sum":298457,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","21"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601748109654324","count":"107","sum":1279668,"bucketCounts":["0","0","0","0","0","0","0","0","19","0","1","17","6","0","0","64"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601748109655684","count":"23","sum":46,"bucketCounts":["0","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601748109655684","count":"107","sum":214,"bucketCounts":["0","107","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601748109657124","count":"23","sum":23,"bucketCounts":["0","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601748109657124","count":"107","sum":107,"bucketCounts":["0","107","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601748109658404","count":"23","sum":23,"bucketCounts":["0","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601748109658404","count":"107","sum":107,"bucketCounts":["0","107","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601749465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601749465000000","asDouble":0.000546052}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601749465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601749465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601749465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601750439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601750439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601750439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601750439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601750439000000","asDouble":0.000628601}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1048576}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":176630}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":418}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0.003792276}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601746.6312335}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601750}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":38400}]}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":5746556}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":26501},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601637}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601750439}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":21389312}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":156316264}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":21},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":109}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":5}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","count":"15","sum":0.001584428,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":938721}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":123}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":8658944}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601636.42}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":20}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":2064384}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000"}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":109}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":14}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":36}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":418}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":112047}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":21},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000"}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":63042824}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601637}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":33}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":2064384}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601751616000000","asDouble":1}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":109}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":19253720}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":19253720}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":1050768}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":685440}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":418}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","count":"109","sum":0,"bucketCounts":["109","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601751616000000","count":"21","sum":157503,"bucketCounts":["0","0","21","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601750}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":85254144}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":26501}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":10000}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0.005115605}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601751616000000","asDouble":1}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":22683648}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0.6}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":825331712}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":1479660}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":44072960}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":31133040}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":8947024}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":46800}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751616000000","asDouble":548160}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601751616000000","count":"109","sum":0.13169711799999995,"bucketCounts":["109","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601751616000000","count":"21","sum":0.04994466100000001,"bucketCounts":["21","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":140173592}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":0.002420445}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":3617},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":3617}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":2.58}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"199","sum":11816,"bucketCounts":["67","3","43","59","4","1","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"178","sum":903667843,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","178"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"110","sum":57145972,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","110"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"178","sum":178,"bucketCounts":["0","178","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"110","sum":0,"bucketCounts":["110","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"48","sum":110598,"bucketCounts":["0","0","0","0","0","0","0","0","0","42","0","0","0","0","0","6"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"178","sum":2592352,"bucketCounts":["0","0","0","0","0","0","0","0","40","0","2","0","6","0","0","130"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"178","sum":178,"bucketCounts":["0","178","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"110","sum":48,"bucketCounts":["62","48","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"24","sum":24,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"109","sum":109,"bucketCounts":["0","109","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":8199},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":8199}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":205520896}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":398}]}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"24","sum":24,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"109","sum":109,"bucketCounts":["0","109","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601751873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601751873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601751873000000","asDouble":0}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"24","sum":312357,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"109","sum":1294146,"bucketCounts":["0","0","0","0","0","0","0","0","20","0","1","17","6","0","0","65"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"24","sum":48,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"109","sum":218,"bucketCounts":["0","109","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601751873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601751873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601751873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601751873000000","asDouble":10000}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":223.76325144499998}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":1129},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":7070}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":569654720}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":199}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"178","sum":356,"bucketCounts":["0","178","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":1}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":8199},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":92441904}]}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"24","sum":6504158,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","24"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","count":"109","sum":19591232,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","109"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601751873000000","asDouble":398}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":3617}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601751873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601753109721089","count":"111","sum":57491057,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","111"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":1121934},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601753109721089","count":"182","sum":927157299,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","182"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601753109724259","count":"182","sum":364,"bucketCounts":["0","182","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601753109726489","count":"182","sum":2684022,"bucketCounts":["0","0","0","0","0","0","0","0","40","0","2","0","6","0","0","134"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601753109726489","count":"49","sum":124552,"bucketCounts":["0","0","0","0","0","0","0","0","0","42","0","0","0","0","0","7"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601753109728879","count":"111","sum":0,"bucketCounts":["111","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601753109728879","count":"182","sum":182,"bucketCounts":["0","182","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601753109729919","count":"111","sum":49,"bucketCounts":["62","49","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601753109729919","count":"182","sum":182,"bucketCounts":["0","182","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601753109730869","count":"112","sum":20004136,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","112"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":84998,"max":1750565},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601753109730869","count":"24","sum":6504158,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","24"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":140568,"max":2029081}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601753109732769","count":"24","sum":312357,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601753109732769","count":"112","sum":1339965,"bucketCounts":["0","0","0","0","0","0","0","0","20","0","1","18","6","0","0","67"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601753109735279","count":"24","sum":48,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601753109735279","count":"112","sum":224,"bucketCounts":["0","112","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601753109736789","count":"112","sum":112,"bucketCounts":["0","112","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601753109736789","count":"24","sum":24,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601753109737699","count":"24","sum":24,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601753109737699","count":"112","sum":112,"bucketCounts":["0","112","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601753109701339","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601753109713759","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601753109709019","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601753109705119","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601753109714409","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601753109710329","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601753109744029","asInt":"3725"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601753109744029","asInt":"3725"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601753109744909","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601753109744909","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601753109746299","asInt":"8255"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601753109746299","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601753109746299","asInt":"8255"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601753109747229","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601753109747229","asInt":"8255"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601753109747229","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601753109749689","asInt":"202"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601753109750159","count":"202","sum":11980,"bucketCounts":["67","3","44","61","4","1","22","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601753109752809","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601753109753749","asInt":"3725"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601753109754619","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601753109755279","asInt":"1129"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601753109755279","asInt":"7126"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601753109756059","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601753109756059","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601753109758429","asDouble":228.76332842099998}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601753109553782","asInt":"95306072"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601753109760179","asInt":"579028824"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601753109558632","asInt":"140173592"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601753109761089","asDouble":2.63}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601753109697529","asInt":"205520896"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601754465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601754465000000","asDouble":0.0006949}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601754465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601754465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601754465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601755439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601755439000000","asDouble":0.00066594}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601755439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601755439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601755439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0.003792276}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":33}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":85254144}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":114}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1480444}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":14}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","count":"114","sum":0.13604762399999995,"bucketCounts":["114","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601756616000000","count":"22","sum":0.05267030100000001,"bucketCounts":["22","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","count":"15","sum":0.001584428,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":5745772}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601637}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":24112768}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":8947024}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":2097152}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":27739}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":22},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":945566}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":114}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601756616000000","asDouble":1}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","count":"114","sum":0,"bucketCounts":["114","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601756616000000","count":"22","sum":165026,"bucketCounts":["0","0","22","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":24112768}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":825331712}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":1}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000"}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":630080}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":63042824}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":20}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":1089678}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":418}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":418}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":183012}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":46800}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":2097152}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":22},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":114}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0.002940197}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601755}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":28524544}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":27739},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":5}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":38400}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":31133040}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":10000}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":128}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601746.6312335}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":685440}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601636.42}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":44040192}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":15515648}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":36}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0.61}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601755}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":161175312}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":8658944}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1700601755439}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000"}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1048576}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":144112}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":0}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756616000000","asDouble":418}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601756616000000","asDouble":1}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601756616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":8573},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":233.764211669}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"51","sum":125844,"bucketCounts":["0","0","0","0","0","0","0","0","0","44","0","0","0","0","0","7"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"186","sum":2713290,"bucketCounts":["0","0","0","0","0","0","0","0","42","0","2","0","6","0","0","136"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601756873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601756873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601756873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601756873000000","asDouble":10000}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":3784},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":3784}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"25","sum":25,"bucketCounts":["0","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"114","sum":114,"bucketCounts":["0","114","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":1}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":205520896}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":588498976}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"186","sum":186,"bucketCounts":["0","186","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"115","sum":0,"bucketCounts":["115","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"25","sum":6676826,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","25"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"114","sum":20265792,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","114"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"25","sum":50,"bucketCounts":["0","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"114","sum":228,"bucketCounts":["0","114","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":101912056}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"208","sum":12357,"bucketCounts":["70","3","45","62","4","1","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"186","sum":372,"bucketCounts":["0","186","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"25","sum":326257,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","23"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"114","sum":1354529,"bucketCounts":["0","0","0","0","0","0","0","0","21","0","1","18","6","0","0","68"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"25","sum":25,"bucketCounts":["0","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"114","sum":114,"bucketCounts":["0","114","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601756873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601756873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601756873000000","asDouble":0}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":208}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":1179},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":7394}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"186","sum":946691343,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","186"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"115","sum":59723575,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","115"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":0.00343956}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":8573},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":8573}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":140173592}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":6}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601756873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":3784}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"186","sum":186,"bucketCounts":["0","186","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","count":"115","sum":51,"bucketCounts":["64","51","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601756873000000","asDouble":2.67}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601758109626088","asDouble":238.764102578}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601758109441011","asInt":"104789944"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601758109631008","asInt":"597982848"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601758109444961","asInt":"140173592"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601758109632598","asDouble":2.72}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601758109604069","asInt":"205520896"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601758109639088","count":"190","sum":965520667,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","190"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":16407354},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601758109639088","count":"116","sum":60161699,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","116"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":1121934}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601758109642258","count":"190","sum":380,"bucketCounts":["0","190","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601758109643688","count":"190","sum":2796582,"bucketCounts":["0","0","0","0","0","0","0","0","42","0","2","0","6","0","0","140"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601758109643688","count":"52","sum":139798,"bucketCounts":["0","0","0","0","0","0","0","0","0","44","0","0","0","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601758109645518","count":"116","sum":0,"bucketCounts":["116","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601758109645518","count":"190","sum":190,"bucketCounts":["0","190","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601758109648118","count":"116","sum":52,"bucketCounts":["64","52","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601758109648118","count":"190","sum":190,"bucketCounts":["0","190","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601758109649518","count":"25","sum":6676826,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","25"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":140568,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601758109649518","count":"117","sum":20654665,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","117"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601758109652148","count":"25","sum":326257,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","23"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601758109652148","count":"117","sum":1396161,"bucketCounts":["0","0","0","0","0","0","0","0","21","0","1","18","7","0","0","70"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601758109653798","count":"117","sum":234,"bucketCounts":["0","117","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601758109653798","count":"25","sum":50,"bucketCounts":["0","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601758109655758","count":"25","sum":25,"bucketCounts":["0","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601758109655758","count":"117","sum":117,"bucketCounts":["0","117","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601758109656738","count":"25","sum":25,"bucketCounts":["0","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601758109656738","count":"117","sum":117,"bucketCounts":["0","117","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601758109609129","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601758109622998","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601758109617978","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601758109613139","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601758109623808","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601758109619258","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601758109661468","asInt":"3882"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601758109661468","asInt":"3882"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601758109662608","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601758109662608","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601758109664398","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601758109664398","asInt":"8629"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601758109664398","asInt":"8629"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601758109665288","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601758109665288","asInt":"8629"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601758109665288","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601758109667708","asInt":"211"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601758109668258","count":"211","sum":12511,"bucketCounts":["70","3","46","64","4","1","23","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601758109670958","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601758109671918","asInt":"3882"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601758109672768","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601758109673518","asInt":"1179"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601758109673518","asInt":"7450"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601758109674458","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601758109674458","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601759465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601759465000000","asDouble":0.00073857}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601759465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601759465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601759465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601760439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601760439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601760439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601760439000000","asDouble":0.000540312}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601760439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601760439}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":167773576}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":44072960}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1481164}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0.65}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":418}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":1035538}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601756.8878708}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":1136898}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601636.42}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601761616000000","asDouble":1}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":521440}]}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":36}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":20}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":33}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":5745052}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","count":"119","sum":0.14272501699999993,"bucketCounts":["119","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601761616000000","count":"23","sum":0.05429762600000001,"bucketCounts":["23","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":418}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":418}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":9740288}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":30308240}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000"}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":2064384}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":133}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":23},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601761616000000","asDouble":1}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":84414464}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":38400}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":46800}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","count":"119","sum":0,"bucketCounts":["119","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601761616000000","count":"23","sum":172552,"bucketCounts":["0","0","23","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601637}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":101360}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1048576}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000"}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":119}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":17088696}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":28977},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":5}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","count":"16","sum":0.001679197,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000059438},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":8947024}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":23552000}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":17088696}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601637}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":20520960}]}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":825331712}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601760}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":685440}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":119}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":189481}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":63042824}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":23},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":119}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":28977}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":2064384}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":14}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1700601760}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0.003487639}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":10000}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601761616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":243.76420007899998}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":1229},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":7718}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"26","sum":340157,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","1","0","24"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"119","sum":1410639,"bucketCounts":["0","0","0","0","0","0","0","0","22","0","1","18","7","0","0","71"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601761873000000","asDouble":1}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":8947},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":206045184}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":1}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":58698576}]}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"194","sum":194,"bucketCounts":["0","194","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"120","sum":54,"bucketCounts":["66","54","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"26","sum":52,"bucketCounts":["0","26","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"119","sum":238,"bucketCounts":["0","119","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"26","sum":26,"bucketCounts":["0","26","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"119","sum":119,"bucketCounts":["0","119","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":0.002952796}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":3940},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":3940}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":140239128}]}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"26","sum":6877544,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","26"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"119","sum":20917302,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","119"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"26","sum":26,"bucketCounts":["0","26","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"119","sum":119,"bucketCounts":["0","119","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601761873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601761873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601761873000000","asDouble":10000}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"194","sum":983040800,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","194"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"120","sum":62358346,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","120"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":6}]}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"194","sum":194,"bucketCounts":["0","194","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"120","sum":0,"bucketCounts":["120","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":8947},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":8947}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"217","sum":12887,"bucketCounts":["73","3","47","65","4","1","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":608170288}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":217}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":3940}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"54","sum":141090,"bucketCounts":["0","0","0","0","0","0","0","0","0","46","0","0","0","0","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"194","sum":2825674,"bucketCounts":["0","0","0","0","0","0","0","0","44","0","2","0","6","0","0","142"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":398}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601761873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601761873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601761873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601761873000000","asDouble":0}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","asDouble":2.79}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601761873000000","count":"194","sum":388,"bucketCounts":["0","194","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"process.runtime.description","value":{"stringValue":"go version go1.21.3 linux/amd64"}},{"key":"process.runtime.name","value":{"stringValue":"go"}},{"key":"process.runtime.version","value":{"stringValue":"go1.21.3"}},{"key":"service.name","value":{"stringValue":"go-faster.oteldb.chotel"}},{"key":"telemetry.sdk.language","value":{"stringValue":"go"}},{"key":"telemetry.sdk.name","value":{"stringValue":"opentelemetry"}},{"key":"telemetry.sdk.version","value":{"stringValue":"1.21.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/runtime","version":"0.46.0"},"metrics":[{"name":"runtime.uptime","description":"Milliseconds since application was initialized","unit":"ms","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917005741","timeUnixNano":"1700601761917231537","asInt":"120000"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.goroutines","description":"Number of goroutines that currently exist","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917011171","timeUnixNano":"1700601761917233947","asInt":"25"}],"aggregationTemporality":2}},{"name":"process.runtime.go.cgo.calls","description":"Number of cgo calls made by the current process","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917013591","timeUnixNano":"1700601761917234387","asInt":"0"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_alloc","description":"Bytes of allocated heap objects","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917028410","timeUnixNano":"1700601761917234837","asInt":"3700432"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_idle","description":"Bytes in idle (unused) spans","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917032650","timeUnixNano":"1700601761917235327","asInt":"9551872"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_inuse","description":"Bytes in in-use spans","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917034590","timeUnixNano":"1700601761917235527","asInt":"5554176"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_objects","description":"Number of allocated heap objects","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917036120","timeUnixNano":"1700601761917235787","asInt":"5305"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_released","description":"Bytes of idle spans whose physical memory has been returned to the OS","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917038040","timeUnixNano":"1700601761917235967","asInt":"6823936"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.heap_sys","description":"Bytes of heap memory obtained from the OS","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917042550","timeUnixNano":"1700601761917236257","asInt":"15106048"}],"aggregationTemporality":2}},{"name":"process.runtime.go.mem.lookups","description":"Number of pointer lookups performed by the runtime","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917056380","timeUnixNano":"1700601761917236547","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.mem.live_objects","description":"Number of live objects is the number of cumulative Mallocs - Frees","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917057580","timeUnixNano":"1700601761917236767","asInt":"5305"}],"aggregationTemporality":2}},{"name":"process.runtime.go.gc.count","description":"Number of completed garbage collection cycles","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917058830","timeUnixNano":"1700601761917236957","asInt":"22"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.gc.pause_total_ns","description":"Cumulative nanoseconds in GC stop-the-world pauses since the program started","sum":{"dataPoints":[{"startTimeUnixNano":"1700601641917064450","timeUnixNano":"1700601761917237157","asInt":"979032"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process.runtime.go.gc.pause_ns","description":"Amount of nanoseconds in GC stop-the-world pauses","histogram":{"dataPoints":[{"startTimeUnixNano":"1700601641917074040","timeUnixNano":"1700601761917237587","count":"22","sum":979032,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","22"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":22100,"max":102909}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.1"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931833929","timeUnixNano":"1700601761917242737","count":"24","sum":13.848560999999998,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0.333605,"max":2.09179}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931840749","timeUnixNano":"1700601761917244387","count":"24","sum":48,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931845139","timeUnixNano":"1700601761917245497","count":"24","sum":560414,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","1","0","0","23"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":4794,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931849989","timeUnixNano":"1700601761917246317","count":"24","sum":24,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601641931852069","timeUnixNano":"1700601761917246967","count":"24","sum":24,"bucketCounts":["0","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.21.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601763110035494","count":"122","sum":63114205,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","122"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":1121934},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601763110035494","count":"198","sum":1005572089,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","198"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":16407354}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601763110038654","count":"198","sum":396,"bucketCounts":["0","198","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601763110042034","count":"198","sum":2918504,"bucketCounts":["0","0","0","0","0","0","0","0","44","0","2","0","6","0","0","146"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601763110042034","count":"55","sum":146300,"bucketCounts":["0","0","0","0","0","0","0","0","0","46","0","0","0","1","0","8"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601763110044114","count":"122","sum":0,"bucketCounts":["122","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601763110044114","count":"198","sum":198,"bucketCounts":["0","198","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601763110045584","count":"122","sum":55,"bucketCounts":["67","55","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601763110045584","count":"198","sum":198,"bucketCounts":["0","198","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601763110046994","count":"27","sum":6986912,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","27"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601763110046994","count":"122","sum":21321217,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","122"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601763110049004","count":"27","sum":345367,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","24"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601763110049004","count":"122","sum":1457038,"bucketCounts":["0","0","0","0","0","0","0","0","22","0","1","19","7","0","0","73"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601763110051104","count":"27","sum":54,"bucketCounts":["0","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601763110051104","count":"122","sum":244,"bucketCounts":["0","122","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601763110052354","count":"27","sum":27,"bucketCounts":["0","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601763110052354","count":"122","sum":122,"bucketCounts":["0","122","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601763110053434","count":"27","sum":27,"bucketCounts":["0","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601763110053434","count":"122","sum":122,"bucketCounts":["0","122","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601763110028994","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601763110024894","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601763110016624","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601763110020904","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601763110029644","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601763110025904","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601763110061844","asInt":"4047"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601763110061844","asInt":"4047"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601763110062864","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601763110062864","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601763110065014","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601763110065014","asInt":"9022"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601763110065014","asInt":"9022"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601763110065954","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601763110065954","asInt":"9022"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601763110065954","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601763110069124","asInt":"221"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601763110069694","count":"221","sum":13069,"bucketCounts":["73","4","48","67","4","1","24","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601763110072904","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601763110074024","asInt":"4047"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601763110074814","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601763110075444","asInt":"7774"},{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601763110075444","asInt":"1248"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601763110076104","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601763110076104","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601763110077224","asDouble":248.764609435}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601763109853537","asInt":"64884128"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601763110078454","asInt":"621665392"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601763109857347","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601763110079144","asDouble":2.86}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601763110012435","asInt":"206045184"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601764465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601764465000000","asDouble":0.000741459}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601764465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601764465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601764465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601765439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601765439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601765439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601765439000000","asDouble":0.00062791}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601765439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601765439}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1481164}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":27262976}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":36}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":14}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":24},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":125}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":685440}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","count":"125","sum":0,"bucketCounts":["125","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601766616000000","count":"24","sum":180075,"bucketCounts":["0","0","24","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":16777216}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000"}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":139}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":44040192}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":38400}]}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":8947024}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601766616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":30337}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1360}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":23549408}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":8986624}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601766616000000","asDouble":1}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0.004185498}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":20}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","count":"125","sum":0.14950488599999995,"bucketCounts":["125","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601766616000000","count":"24","sum":0.05617010800000001,"bucketCounts":["24","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000"}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":23549408}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0.66}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":174234288}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":137813}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601765}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","count":"16","sum":0.001679197,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000059438},{"quantile":0.5,"value":0.000075919},{"quantile":0.75,"value":0.000154358},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1048576}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":84938752}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":5}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":418}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":613600}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601765}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601636.42}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601637670}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601637}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":5745052}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":2097152}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":30308240}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":825331712}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":196418}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":46800}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":418}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":1042724}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601637670}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":24},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":10000}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":34}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1700601756.8878708}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":2097152}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":125}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":418}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":63042824}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":30337},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":1180537}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601766616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766616000000","asDouble":125}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":398}]}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"28","sum":56,"bucketCounts":["0","28","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"124","sum":248,"bucketCounts":["0","124","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"28","sum":28,"bucketCounts":["0","28","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"124","sum":124,"bucketCounts":["0","124","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":4123},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":4123}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":206045184}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":253.764308476}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"227","sum":13463,"bucketCounts":["76","4","49","68","4","1","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"202","sum":202,"bucketCounts":["0","202","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"126","sum":58,"bucketCounts":["68","58","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601766873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601766873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601766873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":9340},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"28","sum":359267,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","25"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"124","sum":1475119,"bucketCounts":["0","0","0","0","0","0","0","0","22","0","2","19","7","0","0","74"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":140239128}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":6}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":2.91}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":227}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"202","sum":404,"bucketCounts":["0","202","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601766873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601766873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601766873000000","asDouble":10000}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":9340},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":9340}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601766873000000","asDouble":1}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"202","sum":1034919758,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","202"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"126","sum":66788612,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","126"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":0.002806439}]}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"28","sum":7155610,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","28"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"124","sum":21584344,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","124"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"28","sum":28,"bucketCounts":["0","28","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"124","sum":124,"bucketCounts":["0","124","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":1298},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":8042}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"202","sum":202,"bucketCounts":["0","202","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"126","sum":0,"bucketCounts":["126","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":634498312}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"58","sum":161492,"bucketCounts":["0","0","0","0","0","0","0","0","0","48","0","0","0","1","0","9"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","count":"202","sum":2954854,"bucketCounts":["0","0","0","0","0","0","0","0","44","0","4","0","6","0","0","148"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601766873000000","asDouble":71531496}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601766873000000","asDouble":4123}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601768109436318","asInt":"4225"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601768109438168","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601768109439238","asInt":"1298"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601768109439238","asInt":"8098"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601768109440808","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601768109440808","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601768109442478","asDouble":258.76372858900004}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601768109277180","asInt":"75245752"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601768109445778","asInt":"644859936"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601768109280780","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601768109447038","asDouble":2.95}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601768109416388","asInt":"206307328"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601768109451598","count":"127","sum":67493951,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","127"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601768109451598","count":"206","sum":1059159373,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","206"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601768109453788","count":"206","sum":412,"bucketCounts":["0","206","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601768109455588","count":"206","sum":3042126,"bucketCounts":["0","0","0","0","0","0","0","0","44","0","4","0","6","0","0","152"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601768109455588","count":"58","sum":161492,"bucketCounts":["0","0","0","0","0","0","0","0","0","48","0","0","0","1","0","9"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601768109457288","count":"127","sum":0,"bucketCounts":["127","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601768109457288","count":"206","sum":206,"bucketCounts":["0","206","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601768109458328","count":"127","sum":58,"bucketCounts":["69","58","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601768109458328","count":"206","sum":206,"bucketCounts":["0","206","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601768109459288","count":"28","sum":7155610,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","28"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601768109459288","count":"127","sum":22053027,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","127"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601768109461168","count":"28","sum":359267,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","25"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601768109461168","count":"127","sum":1518739,"bucketCounts":["0","0","0","0","0","0","0","0","22","0","2","20","7","0","0","76"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601768109462768","count":"28","sum":56,"bucketCounts":["0","28","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601768109462768","count":"127","sum":254,"bucketCounts":["0","127","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601768109463978","count":"28","sum":28,"bucketCounts":["0","28","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601768109463978","count":"127","sum":127,"bucketCounts":["0","127","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601768109465058","count":"127","sum":127,"bucketCounts":["0","127","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601768109465058","count":"28","sum":28,"bucketCounts":["0","28","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601768109432588","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601768109428148","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601768109420028","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601768109423718","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601768109433258","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601768109429168","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601768109469928","asInt":"4225"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601768109469928","asInt":"4225"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601768109470548","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601768109470548","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601768109472318","asInt":"9396"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601768109472318","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601768109472318","asInt":"9396"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601768109473128","asInt":"9396"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601768109473128","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601768109473128","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601768109475358","asInt":"230"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601768109475868","count":"230","sum":13621,"bucketCounts":["76","4","50","70","4","1","25","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601768109478658","asInt":"6"}],"aggregationTemporality":2}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601769465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601769465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601769465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601769465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601769465000000","asDouble":0.000575072}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601770439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601770439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601770439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601770439000000","asDouble":0.000565702}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601770439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":10000}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":2064384}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601770}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":22740992}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":418}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","count":"17","sum":0.001735896,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":5740580}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":202817}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":17901000}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":685440}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1048576}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":825331712}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":17901000}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":20}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":1124768}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0.69}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601770439}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":130}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601771616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601770}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":31575},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":145}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000"}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601637670}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":1226309}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":30902984}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":25},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":130}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":101541}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601766.9337823}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":544960}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":130}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":38400}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":6}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":21331968}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":2064384}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":25},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0.003757395}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0.003792276}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":418}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":63042824}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":84631552}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":33}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1481540}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":44072960}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1}]}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":181209712}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":36}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601636.42}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":418}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","count":"130","sum":0.15603328999999994,"bucketCounts":["130","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601771616000000","count":"25","sum":0.05851571300000001,"bucketCounts":["25","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":31575}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":9437184}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":14}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","count":"130","sum":0,"bucketCounts":["130","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601771616000000","count":"25","sum":187596,"bucketCounts":["0","0","25","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000"}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":0}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":46800}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601771616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":1}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771616000000","asDouble":8951120}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601771616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":140239128}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":236}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"210","sum":210,"bucketCounts":["0","210","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"131","sum":0,"bucketCounts":["131","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":655898360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":6}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":4278}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601771873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601771873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601771873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601771873000000","asDouble":10000}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601771873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601771873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601771873000000","asDouble":0}]}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"210","sum":420,"bucketCounts":["0","210","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":206307328}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"236","sum":13992,"bucketCounts":["79","4","51","71","4","1","26","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"29","sum":58,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"129","sum":258,"bucketCounts":["0","129","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":1}]}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":9714},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":9714}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":263.763731499}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"210","sum":1082784556,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","210"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"131","sum":69269696,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","131"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":4278},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":4278}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":82569920}]}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"29","sum":29,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"129","sum":129,"bucketCounts":["0","129","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":398}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":398}]}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"210","sum":210,"bucketCounts":["0","210","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"131","sum":61,"bucketCounts":["70","61","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"29","sum":7289518,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","29"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"129","sum":22335553,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","129"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"61","sum":176684,"bucketCounts":["0","0","0","0","0","0","0","0","0","50","0","0","0","1","0","10"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"210","sum":3068576,"bucketCounts":["0","0","0","0","0","0","0","0","46","0","4","0","6","0","0","154"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"29","sum":373167,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","26"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"129","sum":1531902,"bucketCounts":["0","0","0","0","0","0","0","0","23","0","2","20","7","0","0","77"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"29","sum":29,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","count":"129","sum":129,"bucketCounts":["0","129","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601771873000000","asDouble":0.00270046}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":9714},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":1348},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601771873000000","asDouble":8366}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601773109696584","asInt":"4401"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601773109696584","asInt":"4401"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601773109698844","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601773109698844","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601773109700434","asInt":"9770"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601773109700434","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601773109700434","asInt":"9770"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601773109702144","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601773109702144","asInt":"9770"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601773109702144","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601773109705214","asInt":"239"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601773109706184","count":"239","sum":14171,"bucketCounts":["79","4","52","72","5","1","26","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601773109709814","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601773109711824","asInt":"4401"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601773109712564","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601773109713174","asInt":"1348"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601773109713174","asInt":"8422"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601773109713884","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601773109713884","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601773109715004","asDouble":268.763986176}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601773109528937","asInt":"85487400"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601773109717954","asInt":"666140008"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601773109534407","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601773109718704","asDouble":3.05}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601773109675835","asInt":"206307328"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601773109723034","count":"132","sum":69592941,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","132"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601773109723034","count":"214","sum":1105528412,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","214"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601773109725184","count":"214","sum":428,"bucketCounts":["0","214","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601773109726214","count":"214","sum":3172178,"bucketCounts":["0","0","0","0","0","0","0","0","46","0","4","0","6","0","0","158"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601773109726214","count":"62","sum":190638,"bucketCounts":["0","0","0","0","0","0","0","0","0","50","0","0","0","1","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601773109727774","count":"132","sum":0,"bucketCounts":["132","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601773109727774","count":"214","sum":214,"bucketCounts":["0","214","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601773109728844","count":"132","sum":62,"bucketCounts":["70","62","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601773109728844","count":"214","sum":214,"bucketCounts":["0","214","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601773109729784","count":"29","sum":7289518,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","29"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601773109729784","count":"132","sum":22832077,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","132"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601773109731414","count":"29","sum":373167,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","26"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601773109731414","count":"132","sum":1583687,"bucketCounts":["0","0","0","0","0","0","0","0","23","0","2","20","8","0","0","79"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601773109732984","count":"132","sum":264,"bucketCounts":["0","132","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601773109732984","count":"29","sum":58,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601773109734014","count":"29","sum":29,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601773109734014","count":"132","sum":132,"bucketCounts":["0","132","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601773109734954","count":"29","sum":29,"bucketCounts":["0","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601773109734954","count":"132","sum":132,"bucketCounts":["0","132","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601773109688004","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601773109680314","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601773109692454","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601773109689084","asInt":"10000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601773109683904","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601773109693104","asInt":"1000"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601774465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601774465000000","asDouble":0.000783688}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601774465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601774465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601774465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601775439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601775439000000","asDouble":0.000547562}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601775439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601775439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601775439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601775}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":32813}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":2097152}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601636.42}]}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":6}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":23777696}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":16678912}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":44040192}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":84631552}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":23777696}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":30902984}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":209281}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":616640}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":150}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":5740580}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0.00343105}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":20}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":1131809}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":135}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","count":"135","sum":0,"bucketCounts":["135","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601776616000000","count":"26","sum":195125,"bucketCounts":["0","0","26","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":1267191}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":8951120}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000"}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1048576}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","count":"135","sum":0.16062533199999995,"bucketCounts":["135","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601776616000000","count":"26","sum":0.060589692000000014,"bucketCounts":["26","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":2097152}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601775439}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":32813},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":9437184}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":685440}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":10000}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":135}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":14}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1360}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":418}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":38400}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":26},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":135}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601776616000000","asDouble":1}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":825331712}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":187086408}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":135382}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0.7}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":418}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":418}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1481540}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601637670}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":33}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000"}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","count":"17","sum":0.001735896,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":27361280}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601766.9337823}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":63042824}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":36}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601776616000000","asDouble":1}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":26},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601775}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":46800}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601776616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776616000000","asDouble":1700601637}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601776873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601776873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601776873000000","asDouble":10000}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":1398},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":8690}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"218","sum":218,"bucketCounts":["0","218","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"136","sum":0,"bucketCounts":["136","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"218","sum":218,"bucketCounts":["0","218","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"136","sum":64,"bucketCounts":["72","64","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":140239128}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"30","sum":60,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"134","sum":268,"bucketCounts":["0","134","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"30","sum":30,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"134","sum":134,"bucketCounts":["0","134","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":1}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":3.09}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":245}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":4460}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":398}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":675495152}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"218","sum":436,"bucketCounts":["0","218","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"30","sum":387067,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","27"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"134","sum":1598251,"bucketCounts":["0","0","0","0","0","0","0","0","24","0","2","20","8","0","0","80"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"30","sum":30,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"134","sum":134,"bucketCounts":["0","134","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601776873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":0.002382604}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":4460},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":4460}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"218","sum":1127680325,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","218"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"136","sum":71734899,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","136"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"30","sum":7420516,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","30"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"134","sum":23167293,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","134"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601776873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601776873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601776873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":10088},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":273.763364012}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"245","sum":14548,"bucketCounts":["82","4","53","73","5","1","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":398}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":398}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":206307328}]}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"64","sum":191930,"bucketCounts":["0","0","0","0","0","0","0","0","0","52","0","0","0","1","0","11"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","count":"218","sum":3201446,"bucketCounts":["0","0","0","0","0","0","0","0","48","0","4","0","6","0","0","160"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":10088},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601776873000000","asDouble":10088}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601776873000000","asDouble":91925064}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601778109567650","count":"137","sum":72215602,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","137"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601778109567650","count":"222","sum":1152575638,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","222"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601778109571299","count":"222","sum":444,"bucketCounts":["0","222","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601778109573239","count":"65","sum":205884,"bucketCounts":["0","0","0","0","0","0","0","0","0","52","0","0","0","1","0","12"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601778109573239","count":"222","sum":3276882,"bucketCounts":["0","0","0","0","0","0","0","0","48","0","4","0","6","0","0","164"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601778109574749","count":"222","sum":222,"bucketCounts":["0","222","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601778109574749","count":"137","sum":0,"bucketCounts":["137","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601778109576149","count":"222","sum":222,"bucketCounts":["0","222","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601778109576149","count":"137","sum":65,"bucketCounts":["72","65","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601778109577149","count":"30","sum":7420516,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","30"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601778109577149","count":"137","sum":23674525,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","137"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601778109578879","count":"137","sum":1635955,"bucketCounts":["0","0","0","0","0","0","0","0","24","0","2","21","8","0","0","82"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601778109578879","count":"30","sum":387067,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","27"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601778109580309","count":"30","sum":60,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601778109580309","count":"137","sum":274,"bucketCounts":["0","137","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601778109581349","count":"30","sum":30,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601778109581349","count":"137","sum":137,"bucketCounts":["0","137","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601778109582449","count":"30","sum":30,"bucketCounts":["0","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601778109582449","count":"137","sum":137,"bucketCounts":["0","137","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601778109547770","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601778109560020","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601778109555640","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601778109551290","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601778109560680","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601778109556610","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601778109588349","asInt":"4550"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601778109588349","asInt":"4550"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601778109589069","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601778109589069","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601778109590299","asInt":"10144"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601778109590299","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601778109590299","asInt":"10144"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601778109591089","asInt":"10144"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601778109591089","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601778109591089","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601778109593829","asInt":"248"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601778109594409","count":"248","sum":14694,"bucketCounts":["82","4","54","75","5","1","27","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601778109596769","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601778109598899","asInt":"4550"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601778109600039","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601778109600619","asInt":"1398"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601778109600619","asInt":"8746"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601778109601339","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601778109601339","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601778109602529","asDouble":278.763238067}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601778109397892","asInt":"94764128"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601778109603909","asInt":"684771880"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601778109401122","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601778109604569","asDouble":3.14}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601778109543440","asInt":"206307328"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601779465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601779465000000","asDouble":0.00069835}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601779465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601779465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601779465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601780439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601780439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601780439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601780439000000","asDouble":0.000552442}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601780439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":6}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1481540}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601780}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":20}]}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1048576}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":85123072}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":8951120}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":2064384}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":418}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":17699432}]}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":27},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":140}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601780}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":140}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":33}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":155}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":8937472}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":10000}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000"}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":34051},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":36}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1360}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":14}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0.73}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":17699432}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":22904832}]}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":418}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000"}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":100236}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601636.42}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":520320}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":5740580}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0.005022006}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":21168128}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":685440}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":34051}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":38400}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":825331712}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","count":"140","sum":0,"bucketCounts":["140","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601781616000000","count":"27","sum":202658,"bucketCounts":["0","0","27","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":215529}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":1310821}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":44072960}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":193980544}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":1210585}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601776.981948}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601781616000000","asDouble":1}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601780439}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":46800}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","count":"140","sum":0.16730181399999994,"bucketCounts":["140","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601781616000000","count":"27","sum":0.06240450600000001,"bucketCounts":["27","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":140}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601637670}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":2064384}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":418}]}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1700601637}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":31332776}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":0}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":27},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601781616000000","asDouble":1}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":1}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601781616000000","count":"18","sum":0.001794886,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781616000000","asDouble":63042824}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":398}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":101205424}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":4608}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"226","sum":226,"bucketCounts":["0","226","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"141","sum":0,"bucketCounts":["141","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"31","sum":62,"bucketCounts":["0","31","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"139","sum":278,"bucketCounts":["0","139","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":694052240}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"254","sum":15070,"bucketCounts":["85","4","55","76","5","1","28","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601781873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601781873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601781873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601781873000000","asDouble":10000}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601781873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601781873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601781873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":10462},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"68","sum":221076,"bucketCounts":["0","0","0","0","0","0","0","0","0","54","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"226","sum":3305974,"bucketCounts":["0","0","0","0","0","0","0","0","50","0","4","0","6","0","0","166"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":0.002781609}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":140239128}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"226","sum":1175234822,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","226"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"141","sum":74249470,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","141"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"31","sum":400967,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","28"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"139","sum":1650433,"bucketCounts":["0","0","0","0","0","0","0","0","25","0","2","21","8","0","0","83"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"226","sum":452,"bucketCounts":["0","226","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"31","sum":31,"bucketCounts":["0","31","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"139","sum":139,"bucketCounts":["0","139","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":398}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":283.763525553}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":1448},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":9014}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"31","sum":31,"bucketCounts":["0","31","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"139","sum":139,"bucketCounts":["0","139","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"31","sum":7575924,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","31"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"139","sum":23902332,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","139"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":1}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":398}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":4608},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":4608}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":3.1900000000000004}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":206569472}]}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"226","sum":226,"bucketCounts":["0","226","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","count":"141","sum":68,"bucketCounts":["73","68","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":10462},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":10462}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601781873000000","asDouble":6}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":254}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601781873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601783109333355","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601783109344055","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601783109339985","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601783109335685","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601783109344795","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601783109341005","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601783109351065","asInt":"4706"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601783109351065","asInt":"4706"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601783109352905","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601783109352905","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601783109354165","asInt":"10518"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601783109354165","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601783109354165","asInt":"10518"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601783109354955","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601783109354955","asInt":"10518"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601783109354955","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601783109357235","asInt":"257"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601783109358065","count":"257","sum":15224,"bucketCounts":["85","4","56","78","5","1","28","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601783109361465","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601783109362565","asInt":"4706"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601783109363765","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601783109364875","asInt":"1448"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601783109364875","asInt":"9070"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601783109365925","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601783109365925","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601783109368315","asDouble":288.763308178}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601783109175087","asInt":"104084568"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601783109371755","asInt":"703372680"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601783109179357","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601783109372445","asDouble":3.2300000000000004}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601783109330035","asInt":"206569472"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601783109376045","count":"230","sum":1194709804,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","230"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601783109376045","count":"142","sum":74604855,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","142"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601783109377764","count":"230","sum":460,"bucketCounts":["0","230","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601783109379614","count":"230","sum":3389262,"bucketCounts":["0","0","0","0","0","0","0","0","50","0","4","0","6","0","0","170"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601783109379614","count":"68","sum":221076,"bucketCounts":["0","0","0","0","0","0","0","0","0","54","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601783109381034","count":"142","sum":0,"bucketCounts":["142","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601783109381034","count":"230","sum":230,"bucketCounts":["0","230","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601783109382074","count":"142","sum":68,"bucketCounts":["74","68","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601783109382074","count":"230","sum":230,"bucketCounts":["0","230","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601783109383904","count":"31","sum":7575924,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","31"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601783109383904","count":"142","sum":24255066,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","142"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601783109386024","count":"31","sum":400967,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","28"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601783109386024","count":"142","sum":1692063,"bucketCounts":["0","0","0","0","0","0","0","0","25","0","2","21","9","0","0","85"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601783109387384","count":"31","sum":62,"bucketCounts":["0","31","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601783109387384","count":"142","sum":284,"bucketCounts":["0","142","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601783109388404","count":"31","sum":31,"bucketCounts":["0","31","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601783109388404","count":"142","sum":142,"bucketCounts":["0","142","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601783109389274","count":"31","sum":31,"bucketCounts":["0","31","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601783109389274","count":"142","sum":142,"bucketCounts":["0","142","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601784465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601784465000000","asDouble":0.000619361}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601784465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601784465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601784465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601785439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601785439000000","asDouble":0.000646581}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601785439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601785439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601785439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":85909504}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601786616000000","asDouble":1}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":36}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":28},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":145}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":35289}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0.003792276}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":14}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","count":"145","sum":0,"bucketCounts":["145","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601786616000000","count":"28","sum":210180,"bucketCounts":["0","0","28","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601785}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601637670}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601776.981948}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601785439}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":38400}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":2097152}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601636.42}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":16736256}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":8069120}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":31332776}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0.75}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":35289},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":160}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1481916}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":592000}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":1352179}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601786616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1360}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":1217784}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","count":"18","sum":0.001794886,"quantileValues":[{"value":0.00004302},{"quantile":0.25,"value":0.000058859},{"quantile":0.5,"value":0.000068329},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":28},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":8951120}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1048576}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":145}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":685440}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":221916}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601637670}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":5740204}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":2097152}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":199919576}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":134395}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000"}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":418}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":23638464}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":46800}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":6}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0.004000171}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":418}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":63042824}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":10000}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1700601785}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":418}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":33}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":27303936}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":44040192}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":0}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":23638464}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000"}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","count":"145","sum":0.17360371899999996,"bucketCounts":["145","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601786616000000","count":"28","sum":0.06574267800000001,"bucketCounts":["28","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":825331712}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":20}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786616000000","asDouble":145}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601786616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"70","sum":222368,"bucketCounts":["0","0","0","0","0","0","0","0","0","56","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"234","sum":3418530,"bucketCounts":["0","0","0","0","0","0","0","0","52","0","4","0","6","0","0","172"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601786873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601786873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601786873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601786873000000","asDouble":10000}]}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":3.3100000000000005}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":4765}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":293.763556423}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"32","sum":32,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"144","sum":144,"bucketCounts":["0","144","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601786873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601786873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601786873000000","asDouble":0}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":4765},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":4765}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":140239128}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":207618048}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":62784624}]}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"32","sum":64,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"144","sum":288,"bucketCounts":["0","144","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":263}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":1498},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":9338}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"234","sum":234,"bucketCounts":["0","234","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"146","sum":70,"bucketCounts":["76","70","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":0.002552833}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"234","sum":1214057797,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","234"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"146","sum":77863816,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","146"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"234","sum":468,"bucketCounts":["0","234","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"32","sum":7718982,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","32"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"144","sum":24479343,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","144"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"263","sum":15601,"bucketCounts":["88","4","57","79","5","1","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"32","sum":414867,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","29"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"144","sum":1706627,"bucketCounts":["0","0","0","0","0","0","0","0","26","0","2","21","9","0","0","86"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":1}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":10836},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":716180392}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"234","sum":234,"bucketCounts":["0","234","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"146","sum":0,"bucketCounts":["146","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"32","sum":32,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","count":"144","sum":144,"bucketCounts":["0","144","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":10836},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601786873000000","asDouble":10836}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":398}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601786873000000","asDouble":398}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601788109424432","asDouble":298.76361048}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601788109250704","asInt":"66608000"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601788109428892","asInt":"729932336"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601788109253694","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601788109430472","asDouble":3.39}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601788109407382","asInt":"207618048"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601788109436192","count":"147","sum":78356239,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","147"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601788109436192","count":"238","sum":1237497231,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","238"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601788109439032","count":"238","sum":476,"bucketCounts":["0","238","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601788109440352","count":"238","sum":3509280,"bucketCounts":["0","0","0","0","0","0","0","0","52","0","4","0","6","0","0","176"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601788109440352","count":"70","sum":222368,"bucketCounts":["0","0","0","0","0","0","0","0","0","56","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601788109441781","count":"147","sum":0,"bucketCounts":["147","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601788109441781","count":"238","sum":238,"bucketCounts":["0","238","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601788109442841","count":"147","sum":70,"bucketCounts":["77","70","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601788109442841","count":"238","sum":238,"bucketCounts":["0","238","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601788109443941","count":"32","sum":7718982,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","32"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601788109443941","count":"147","sum":24892767,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","147"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601788109446231","count":"32","sum":414867,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","29"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601788109446231","count":"147","sum":1751986,"bucketCounts":["0","0","0","0","0","0","0","0","26","0","2","22","9","0","0","88"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601788109447761","count":"32","sum":64,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601788109447761","count":"147","sum":294,"bucketCounts":["0","147","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601788109448831","count":"32","sum":32,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601788109448831","count":"147","sum":147,"bucketCounts":["0","147","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601788109449661","count":"32","sum":32,"bucketCounts":["0","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601788109449661","count":"147","sum":147,"bucketCounts":["0","147","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601788109410172","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601788109422242","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601788109418042","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601788109422802","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601788109419132","asInt":"10000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601788109413782","asInt":"1000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601788109454501","asInt":"4869"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601788109454501","asInt":"4869"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601788109455321","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601788109455321","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601788109456851","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601788109456851","asInt":"10892"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601788109456851","asInt":"10892"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601788109457791","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601788109457791","asInt":"10892"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601788109457791","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601788109459891","asInt":"266"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601788109460461","count":"266","sum":15761,"bucketCounts":["88","4","58","81","5","1","29","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601788109463271","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601788109463961","asInt":"4869"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601788109464671","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601788109465501","asInt":"1498"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601788109465501","asInt":"9394"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601788109466151","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601788109466151","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601789465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601789465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601789465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601789465000000","asDouble":0.000623301}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601789465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601790439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601790439000000","asDouble":0.000618251}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601790439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601790439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601790439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":63042824}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":150}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":1299367}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":529600}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":825331712}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601790}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":23584768}]}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601790}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000"}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601790439}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":150}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0.8}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":46800}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601791616000000","asDouble":1}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","count":"150","sum":0,"bucketCounts":["150","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601791616000000","count":"29","sum":217705,"bucketCounts":["0","0","29","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":165}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","count":"19","sum":0.001835196,"quantileValues":[{"value":0.00004031},{"quantile":0.25,"value":0.000056829},{"quantile":0.5,"value":0.000059909},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":31133536}]}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":2129920}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":34}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":29},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":18446744073709552000}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":85606400}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000"}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":5739820}]}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601636.42}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":29},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":150}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":16938336}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":16938336}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":38400}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":20}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601787.0338433}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":2129920}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":8675328}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":98479}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":14}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":228327}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601791616000000","asDouble":1}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":37}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601637670}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":418}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":1397846}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":36527}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601791616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1482300}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":685440}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1700601637}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":418}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":8951120}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":10000}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1}]}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1360}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":0.003821693}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","count":"150","sum":0.17941465199999995,"bucketCounts":["150","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601791616000000","count":"29","sum":0.06776409800000001,"bucketCounts":["29","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":36527},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":20422656}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":44007424}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":6}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":418}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":206902016}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601791616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791616000000","asDouble":1048576}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":740330208}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"242","sum":242,"bucketCounts":["0","242","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"151","sum":0,"bucketCounts":["151","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"33","sum":66,"bucketCounts":["0","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"149","sum":298,"bucketCounts":["0","149","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":4927},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":4927}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":73182496}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":272}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":140239128}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":1548},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":9662}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601791873000000","asDouble":1}]}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"72","sum":223660,"bucketCounts":["0","0","0","0","0","0","0","0","0","58","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"242","sum":3538372,"bucketCounts":["0","0","0","0","0","0","0","0","54","0","4","0","6","0","0","178"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"33","sum":7865029,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","33"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"149","sum":25200432,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","149"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601791873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601791873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601791873000000","asDouble":10000}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601791873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601791873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601791873000000","asDouble":0}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"242","sum":1256655677,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","242"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"151","sum":80474167,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","151"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"242","sum":484,"bucketCounts":["0","242","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":207618048}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":4927}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":398}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":0.003270712}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":303.763529698}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":6}]}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"33","sum":33,"bucketCounts":["0","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"149","sum":149,"bucketCounts":["0","149","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"33","sum":33,"bucketCounts":["0","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"149","sum":149,"bucketCounts":["0","149","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":11210},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":11210},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":11210}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":3.44}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"242","sum":242,"bucketCounts":["0","242","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"151","sum":72,"bucketCounts":["79","72","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"272","sum":16137,"bucketCounts":["91","4","59","82","5","1","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"33","sum":428767,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","30"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601791873000000","count":"149","sum":1766464,"bucketCounts":["0","0","0","0","0","0","0","0","27","0","2","22","9","0","0","89"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601791873000000","asDouble":1}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601793110181137","asInt":"275"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601793110183207","count":"275","sum":16283,"bucketCounts":["91","4","60","84","5","1","30","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601793110187177","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601793110188347","asInt":"5017"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601793110189207","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601793110190297","asInt":"1548"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601793110190297","asInt":"9718"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601793110191167","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601793110191167","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601793110192497","asDouble":308.76430261400003}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601793110016600","asInt":"76038776"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601793110195247","asInt":"749760984"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601793110027590","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601793110196147","asDouble":3.49}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601793110160728","asInt":"207880192"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601793110200707","count":"152","sum":80837742,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","152"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601793110200707","count":"246","sum":1275292852,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","246"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601793110203247","count":"246","sum":492,"bucketCounts":["0","246","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601793110204227","count":"246","sum":3613796,"bucketCounts":["0","0","0","0","0","0","0","0","54","0","4","0","6","0","0","182"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601793110204227","count":"72","sum":223660,"bucketCounts":["0","0","0","0","0","0","0","0","0","58","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601793110206097","count":"152","sum":0,"bucketCounts":["152","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601793110206097","count":"246","sum":246,"bucketCounts":["0","246","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601793110207007","count":"152","sum":72,"bucketCounts":["80","72","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601793110207007","count":"246","sum":246,"bucketCounts":["0","246","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601793110208067","count":"33","sum":7865029,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","33"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601793110208067","count":"152","sum":25782153,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","152"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601793110209887","count":"33","sum":428767,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","30"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601793110209887","count":"152","sum":1804162,"bucketCounts":["0","0","0","0","0","0","0","0","27","0","2","23","9","0","0","91"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601793110211367","count":"33","sum":66,"bucketCounts":["0","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601793110211367","count":"152","sum":304,"bucketCounts":["0","152","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601793110212337","count":"33","sum":33,"bucketCounts":["0","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601793110212337","count":"152","sum":152,"bucketCounts":["0","152","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601793110213147","count":"33","sum":33,"bucketCounts":["0","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601793110213147","count":"152","sum":152,"bucketCounts":["0","152","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601793110164578","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601793110177457","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601793110173207","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601793110168667","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601793110178127","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601793110174517","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601793110217647","asInt":"5017"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601793110217647","asInt":"5017"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601793110218357","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601793110218357","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601793110219627","asInt":"11266"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601793110219627","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601793110219627","asInt":"11266"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601793110220257","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601793110220257","asInt":"11266"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601793110220257","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601794465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601794465000000","asDouble":0.000765358}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601794465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601794465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601794465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601795439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601795439000000","asDouble":0.00066838}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601795439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601795439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601795439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":25698304}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000"}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":37765},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1048576}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":2162688}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0.81}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000"}]}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601796616000000","asDouble":1}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":22006232}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601796616000000","asDouble":1}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","count":"155","sum":0.18507699799999994,"bucketCounts":["155","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601796616000000","count":"30","sum":0.069597271,"bucketCounts":["30","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","count":"19","sum":0.001835196,"quantileValues":[{"value":0.00004031},{"quantile":0.25,"value":0.000056829},{"quantile":0.5,"value":0.000059909},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601637}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":33}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":85606400}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":155}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":14}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":20}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":10000}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":7}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":31133536}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":171}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601795}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601795}]}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":418}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":37}]}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":22006232}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":46800}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0.00471661}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":418}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":155}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601637}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":43974656}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":1437243}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":1305880}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":131363}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":18276352}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":6001588}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":37765}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1482676}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601636.42}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":30},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":155}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","count":"155","sum":0,"bucketCounts":["155","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601796616000000","count":"30","sum":225226,"bucketCounts":["0","0","30","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":211969912}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":418}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":234689}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":63304968}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":8675328}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":825593856}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601787.0338433}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":2162688}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":30},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601637670}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":601280}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":685440}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601796616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601796616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1700601795439}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":8951120}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601796616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":1}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796616000000","asDouble":38400}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":0.002763769}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":5076},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":5076}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":207880192}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"34","sum":68,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"154","sum":308,"bucketCounts":["0","154","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601796873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601796873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601796873000000","asDouble":0}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":11584},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":11584},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":11584}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"250","sum":1294406789,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","250"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"156","sum":83333245,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","156"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"74","sum":224952,"bucketCounts":["0","0","0","0","0","0","0","0","0","60","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"250","sum":3643064,"bucketCounts":["0","0","0","0","0","0","0","0","56","0","4","0","6","0","0","184"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"34","sum":442667,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","31"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"154","sum":1818726,"bucketCounts":["0","0","0","0","0","0","0","0","28","0","2","23","9","0","0","92"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":398}]}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"34","sum":34,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"154","sum":154,"bucketCounts":["0","154","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":1598},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":9986}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"250","sum":500,"bucketCounts":["0","250","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"34","sum":8032626,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","34"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"154","sum":26005540,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","154"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":3.54}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":82460696}]}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601796873000000","asDouble":1}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":6}]}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":5076}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"250","sum":250,"bucketCounts":["0","250","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"156","sum":74,"bucketCounts":["82","74","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":759039184}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":140239128}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":313.764380158}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601796873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601796873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601796873000000","asDouble":10000}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":398}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":1}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601796873000000","asDouble":398}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"281","sum":16660,"bucketCounts":["94","4","61","85","5","1","31","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","asDouble":281}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"250","sum":250,"bucketCounts":["0","250","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"156","sum":0,"bucketCounts":["156","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"34","sum":34,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601796873000000","count":"154","sum":154,"bucketCounts":["0","154","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601798109242746","count":"157","sum":83710300,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","157"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601798109242746","count":"254","sum":1325291252,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","254"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601798109247696","count":"254","sum":508,"bucketCounts":["0","254","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601798109250756","count":"254","sum":3746144,"bucketCounts":["0","0","0","0","0","0","0","0","56","0","4","0","6","0","0","188"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601798109250756","count":"74","sum":224952,"bucketCounts":["0","0","0","0","0","0","0","0","0","60","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601798109253946","count":"157","sum":0,"bucketCounts":["157","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601798109253946","count":"254","sum":254,"bucketCounts":["0","254","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601798109255086","count":"157","sum":74,"bucketCounts":["83","74","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601798109255086","count":"254","sum":254,"bucketCounts":["0","254","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601798109256026","count":"34","sum":8032626,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","34"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601798109256026","count":"157","sum":26508482,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","157"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601798109257506","count":"34","sum":442667,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","31"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601798109257506","count":"157","sum":1870250,"bucketCounts":["0","0","0","0","0","0","0","0","28","0","2","23","10","0","0","94"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601798109258676","count":"34","sum":68,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601798109258676","count":"157","sum":314,"bucketCounts":["0","157","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601798109259676","count":"157","sum":157,"bucketCounts":["0","157","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601798109259676","count":"34","sum":34,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601798109260426","count":"34","sum":34,"bucketCounts":["0","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601798109260426","count":"157","sum":157,"bucketCounts":["0","157","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601798109227276","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601798109239256","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601798109235026","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601798109231336","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601798109239806","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601798109236116","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601798109266186","asInt":"5198"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601798109266186","asInt":"5198"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601798109267206","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601798109267206","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601798109268556","asInt":"11640"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601798109268556","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601798109268556","asInt":"11640"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601798109269466","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601798109269466","asInt":"11640"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601798109269466","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601798109271456","asInt":"284"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601798109271946","count":"284","sum":16838,"bucketCounts":["94","4","62","86","6","1","31","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601798109273866","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601798109275146","asInt":"5198"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601798109275886","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601798109276536","asInt":"1598"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601798109276536","asInt":"10042"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601798109277486","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601798109277486","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601798109278326","asDouble":318.763473666}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601798109093218","asInt":"85389488"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601798109279366","asInt":"768389896"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601798109095858","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601798109280526","asDouble":3.59}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601798109223427","asInt":"207880192"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601799465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601799465000000","asDouble":0.00068406}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601799465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601799465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601799465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601800439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601800439000000","asDouble":0.000628541}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601800439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601800439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601800439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601636.42}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":241062}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000"}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","count":"20","sum":0.001875544,"quantileValues":[{"value":0.00004031},{"quantile":0.25,"value":0.000056829},{"quantile":0.5,"value":0.000059909},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":46800}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":219067728}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601637670}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":31},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1360}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":418}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":8953168}]}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":85606400}]}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1}]}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":31047400}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":20}]}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1}]}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":19275776}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","count":"160","sum":0,"bucketCounts":["160","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601801616000000","count":"31","sum":232744,"bucketCounts":["0","0","31","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":10000}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":418}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":418}]}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":1395981}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":24731648}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":39003},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601800}]}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0.003349631}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1483324}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":15311280}]}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":8339456}]}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601801616000000","asDouble":1}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":31},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":160}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601637}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":33}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":37}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":825593856}]}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":7}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1048576}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":63304968}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":160}]}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":160}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":15311280}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000"}]}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601800439}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":176}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":685440}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":2129920}]}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601801616000000","asDouble":1}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":39003}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":5998892}]}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":44007424}]}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601800}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1700601798.2886453}]}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":506400}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":2129920}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":88226}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":38400}]}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":14}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":0}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801616000000","asDouble":1}]}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":1484207}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0.85}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","count":"160","sum":0.19146967499999992,"bucketCounts":["160","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601801616000000","count":"31","sum":0.07233726,"bucketCounts":["31","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601801616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"258","sum":258,"bucketCounts":["0","258","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"161","sum":76,"bucketCounts":["85","76","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"35","sum":8203173,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","35"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"159","sum":26771769,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","159"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":11958},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":398}]}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"35","sum":35,"bucketCounts":["0","35","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"159","sum":159,"bucketCounts":["0","159","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":777768424}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":140239128}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":1648},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":10310}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"35","sum":70,"bucketCounts":["0","35","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"159","sum":318,"bucketCounts":["0","159","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601801873000000","asDouble":1}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601801873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601801873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601801873000000","asDouble":0}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":91839224}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":0.002977836}]}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":207880192}]}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"258","sum":516,"bucketCounts":["0","258","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"76","sum":226244,"bucketCounts":["0","0","0","0","0","0","0","0","0","62","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"258","sum":3775236,"bucketCounts":["0","0","0","0","0","0","0","0","58","0","4","0","6","0","0","190"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"258","sum":1354566720,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","258"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"161","sum":85746171,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","161"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":11958},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":11958}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"290","sum":17214,"bucketCounts":["97","4","63","87","6","1","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":290}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":5256}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"258","sum":258,"bucketCounts":["0","258","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"161","sum":0,"bucketCounts":["161","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601801873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601801873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601801873000000","asDouble":10000}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":1}]}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"35","sum":35,"bucketCounts":["0","35","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"159","sum":159,"bucketCounts":["0","159","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":6}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601801873000000","asDouble":398}]}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":323.763186925}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":3.63}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"35","sum":456567,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","32"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","count":"159","sum":1884728,"bucketCounts":["0","0","0","0","0","0","0","0","29","0","2","23","10","0","0","95"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":5256},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601801873000000","asDouble":5256}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601803110117758","asDouble":328.76404570700004}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601803109956030","asInt":"94698456"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601803110121898","asInt":"787077392"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601803109959580","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601803110123958","asDouble":3.67}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601803110099138","asInt":"207880192"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601803110128978","count":"262","sum":1374513132,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","262"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601803110128978","count":"162","sum":86123666,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","162"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601803110132108","count":"262","sum":524,"bucketCounts":["0","262","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601803110133758","count":"262","sum":3862844,"bucketCounts":["0","0","0","0","0","0","0","0","58","0","4","0","6","0","0","194"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601803110133758","count":"76","sum":226244,"bucketCounts":["0","0","0","0","0","0","0","0","0","62","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601803110135848","count":"162","sum":0,"bucketCounts":["162","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601803110135848","count":"262","sum":262,"bucketCounts":["0","262","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601803110136938","count":"162","sum":76,"bucketCounts":["86","76","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601803110136938","count":"262","sum":262,"bucketCounts":["0","262","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601803110139018","count":"35","sum":8203173,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","35"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601803110139018","count":"162","sum":27194033,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","162"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601803110141248","count":"35","sum":456567,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","32"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601803110141248","count":"162","sum":1928516,"bucketCounts":["0","0","0","0","0","0","0","0","29","0","2","23","11","0","0","97"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601803110142808","count":"35","sum":70,"bucketCounts":["0","35","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601803110142808","count":"162","sum":324,"bucketCounts":["0","162","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601803110143808","count":"35","sum":35,"bucketCounts":["0","35","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601803110143808","count":"162","sum":162,"bucketCounts":["0","162","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601803110144628","count":"35","sum":35,"bucketCounts":["0","35","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601803110144628","count":"162","sum":162,"bucketCounts":["0","162","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601803110103148","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601803110115168","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601803110111038","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601803110106988","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601803110115808","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601803110112028","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601803110148977","asInt":"5357"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601803110148977","asInt":"5357"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601803110149637","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601803110149637","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601803110150947","asInt":"12014"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601803110150947","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601803110150947","asInt":"12014"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601803110151637","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601803110151637","asInt":"12014"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601803110151637","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601803110154197","asInt":"293"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601803110154797","count":"293","sum":17371,"bucketCounts":["97","4","64","89","6","1","32","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601803110157337","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601803110158057","asInt":"5357"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601803110158937","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601803110159607","asInt":"1648"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601803110159607","asInt":"10366"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601803110160387","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601803110160387","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601804465000000","asDouble":0.000603611}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601804465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601804465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601804465000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601804465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601805439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601805439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601805439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601805439000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601805439000000","asDouble":0.00066069}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":1402485}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":7}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":24838144}]}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","count":"165","sum":0.1975055259999999,"bucketCounts":["165","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601806616000000","count":"32","sum":0.074106274,"bucketCounts":["32","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1}]}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":21105760}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":7970816}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":37}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":40241},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":33}]}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":247390}]}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":165}]}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":10000}]}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601637670}]}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":43974656}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601636.42}]}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":31047400}]}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":181}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0.00400145}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":825593856}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000"}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0.003792276}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","count":"20","sum":0.001875544,"quantileValues":[{"value":0.00004031},{"quantile":0.25,"value":0.000056829},{"quantile":0.5,"value":0.000059909},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":8953168}]}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":418}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":32},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":165}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601805}]}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000"}]}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1483324}]}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":1522835}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":557600}]}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1048576}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":46800}]}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601805439}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":32},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601805}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":418}]}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":418}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":224862208}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":5998892}]}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":14}]}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1360}]}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601806616000000","asDouble":1}]}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":21105760}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":19136512}]}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":120350}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":685440}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":2162688}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":63304968}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","count":"165","sum":0,"bucketCounts":["165","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601806616000000","count":"32","sum":240264,"bucketCounts":["0","0","32","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601806616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":85868544}]}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0.86}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":38400}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1360}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":2162688}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":20}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":40241}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":165}]}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1700601798.2886453}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":1}]}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601806616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":12332},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":12332}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":5416},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":5416}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":207880192}]}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"36","sum":8388621,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","36"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"164","sum":27436270,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","164"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":12332},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":3.71}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":333.764483485}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"266","sum":532,"bucketCounts":["0","266","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"78","sum":227536,"bucketCounts":["0","0","0","0","0","0","0","0","0","64","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"266","sum":3892112,"bucketCounts":["0","0","0","0","0","0","0","0","60","0","4","0","6","0","0","196"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"36","sum":36,"bucketCounts":["0","36","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"164","sum":164,"bucketCounts":["0","164","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601806873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601806873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601806873000000","asDouble":0}]}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":101949048}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":1698},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":10634}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":5416}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"36","sum":72,"bucketCounts":["0","36","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"164","sum":328,"bucketCounts":["0","164","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":0.00405786}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":398}]}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"299","sum":17748,"bucketCounts":["100","4","65","90","6","1","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":299}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"36","sum":470467,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","33"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"164","sum":1943080,"bucketCounts":["0","0","0","0","0","0","0","0","30","0","2","23","11","0","0","98"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":140239128}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":6}]}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"266","sum":266,"bucketCounts":["0","266","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"166","sum":0,"bucketCounts":["166","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"266","sum":266,"bucketCounts":["0","266","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"166","sum":78,"bucketCounts":["88","78","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":797187216}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":398}]}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601806873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601806873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601806873000000","asDouble":10000}]}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"36","sum":36,"bucketCounts":["0","36","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"164","sum":164,"bucketCounts":["0","164","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601806873000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":398}]}},{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"266","sum":1392611084,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","266"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601806873000000","count":"166","sum":87821710,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","166"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601806873000000","asDouble":1}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601808109855006","asInt":"5504"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601808109855006","asInt":"5504"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601808109857716","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601808109857716","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601808109859316","asInt":"12388"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601808109859316","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601808109859316","asInt":"12388"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601808109860796","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601808109860796","asInt":"12388"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601808109860796","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601808109863306","asInt":"302"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601808109864446","count":"302","sum":17892,"bucketCounts":["100","4","66","92","6","1","33","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601808109867526","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601808109868846","asInt":"5504"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601808109869496","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601808109870216","asInt":"1698"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601808109870216","asInt":"10690"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601808109870946","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601808109870946","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601808109872006","asDouble":338.764230883}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601808109703209","asInt":"105663344"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601808109874376","asInt":"808152104"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601808109706709","asInt":"140239128"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601808109875216","asDouble":3.7600000000000002}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601808109835977","asInt":"208666624"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601808109879966","count":"167","sum":88199824,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","167"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601808109879966","count":"270","sum":1409645901,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","270"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601808109882046","count":"270","sum":540,"bucketCounts":["0","270","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601808109883306","count":"270","sum":3968808,"bucketCounts":["0","0","0","0","0","0","0","0","60","0","4","0","6","0","0","200"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601808109883306","count":"78","sum":227536,"bucketCounts":["0","0","0","0","0","0","0","0","0","64","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601808109884596","count":"167","sum":0,"bucketCounts":["167","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601808109884596","count":"270","sum":270,"bucketCounts":["0","270","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601808109885626","count":"167","sum":78,"bucketCounts":["89","78","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601808109885626","count":"270","sum":270,"bucketCounts":["0","270","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601808109886556","count":"36","sum":8388621,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","36"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601808109886556","count":"167","sum":27824445,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","167"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601808109888696","count":"36","sum":470467,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","33"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601808109888696","count":"167","sum":1981414,"bucketCounts":["0","0","0","0","0","0","0","0","30","0","2","24","11","0","0","100"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601808109890326","count":"167","sum":334,"bucketCounts":["0","167","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601808109890326","count":"36","sum":72,"bucketCounts":["0","36","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601808109892036","count":"36","sum":36,"bucketCounts":["0","36","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601808109892036","count":"167","sum":167,"bucketCounts":["0","167","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601808109893286","count":"36","sum":36,"bucketCounts":["0","36","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601808109893286","count":"167","sum":167,"bucketCounts":["0","167","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601808109840127","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601808109852466","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601808109847846","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601808109843616","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601808109853136","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601808109849316","asInt":"10000"}]}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"oteldb"}},{"key":"net.host.name","value":{"stringValue":"oteldb"}},{"key":"service.instance.id","value":{"stringValue":"oteldb:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601809465000000","asDouble":0}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601809465000000","asDouble":0.000628251}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601809465000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601809465000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601809465000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"clickhouse"}},{"key":"net.host.name","value":{"stringValue":"clickhouse"}},{"key":"service.instance.id","value":{"stringValue":"clickhouse:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601810439000000","asDouble":0.000842237}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601810439000000","asDouble":0}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601810439000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601810439000000","asDouble":0}]}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601810439000000","asDouble":0}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"prometheus"}},{"key":"net.host.name","value":{"stringValue":"prometheus"}},{"key":"service.instance.id","value":{"stringValue":"prometheus:9090"}},{"key":"net.host.port","value":{"stringValue":"9090"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"go_memstats_next_gc_bytes","description":"Number of heap bytes when next garbage collection will take place.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":31047400}]}},{"name":"prometheus_http_requests_total","description":"Counter of HTTP requests.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/healthy"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/quit"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/ready"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/-/reload"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/*path"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/clean_tombstones"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/delete_series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/admin/tsdb/snapshot"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alertmanagers"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/alerts"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/format_query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/label/:name/values"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/labels"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_exemplars"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/query_range"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/read"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/scrape_pools"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/series"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/buildinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/runtimeinfo"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/tsdb"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/status/walreplay"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/targets/metadata"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/classic/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/config"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/consoles/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/debug/*subpath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/favicon.ico"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/federate"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/flags"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/graph"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/manifest.json"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":33},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/rules"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/service-discovery"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/starting"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/static/*filepath"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/targets"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/tsdb-status"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"200"}},{"key":"handler","value":{"stringValue":"/version"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"204"}},{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":170}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansions_total","description":"The total number of template text expansions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_skipped_total","description":"Total number of skipped compactions due to disabled auto compaction.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks","description":"Total number of chunks in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1360}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":418}]}},{"name":"go_gc_duration_seconds","description":"A summary of the pause duration of garbage collection cycles.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","count":"20","sum":0.001875544,"quantileValues":[{"value":0.00004031},{"quantile":0.25,"value":0.000056829},{"quantile":0.5,"value":0.000059909},{"quantile":0.75,"value":0.000107569},{"quantile":1,"value":0.000255906}]}]}},{"name":"net_conntrack_dialer_conn_failed_total","description":"Total number of connections failed to dial by the dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"refused"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"resolution"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"timeout"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}},{"key":"reason","value":{"stringValue":"unknown"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_config_last_reload_success_timestamp_seconds","description":"Timestamp of the last successful configuration reload.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601637.3187819}]}},{"name":"prometheus_ready","description":"Whether Prometheus startup was fully completed and the server is ready for normal operation.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1}]}},{"name":"prometheus_target_scrape_pool_exceeded_label_limits_total","description":"Total number of times scrape pools hit the label limits, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series","description":"Total number of series in the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1360}]}},{"name":"promhttp_metric_handler_requests_in_flight","description":"Current number of scrapes being served.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1}]}},{"name":"process_cpu_seconds_total","description":"Total user and system CPU time spent in seconds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0.87}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_duration_seconds","description":"Query timings","summary":{"dataPoints":[{"attributes":[{"key":"slice","value":{"stringValue":"inner_eval"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"prepare_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"queue_time"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"slice","value":{"stringValue":"result_sort"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_remote_storage_samples_in_total","description":"Samples in to remote storage, compare to samples out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":41479}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_rule_group_duration_seconds","description":"The duration of rule group evaluations.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.01,"value":"NaN"},{"quantile":0.05,"value":"NaN"},{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_http_failures_total","description":"Number of HTTP service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_max_exemplars","description":"Total number of exemplars the exemplar storage can store, resizeable.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"go_goroutines","description":"Number of goroutines that currently exist.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":33}]}},{"name":"prometheus_tsdb_head_max_time_seconds","description":"Maximum timestamp of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601810}]}},{"name":"prometheus_tsdb_head_out_of_order_samples_appended_total","description":"Total number of appended out of order samples.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_chunks_storage_size_bytes","description":"Size of the chunks_head directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_truncate_duration_seconds","description":"Duration of WAL truncation.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000"}]}},{"name":"prometheus_tsdb_wal_truncations_failed_total","description":"Total number of write log truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_populating_block","description":"Set to 1 when a block is currently being written to the disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_min_time_seconds","description":"Minimum time bound of the head block.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601637}]}},{"name":"prometheus_tsdb_head_series_created_total","description":"Total number of series created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_high_watermark","description":"The highest TSDB append ID that has been given out.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":170}]}},{"name":"prometheus_tsdb_lowest_timestamp_seconds","description":"Lowest timestamp value stored in the database.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601637}]}},{"name":"prometheus_sd_failed_configs","description":"Current number of service discovery configurations that failed to load.","gauge":{"dataPoints":[{"attributes":[{"key":"name","value":{"stringValue":"notify"}}],"timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"name","value":{"stringValue":"scrape"}}],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_target_scrapes_exemplar_out_of_order_total","description":"Total number of exemplar rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_order_total","description":"Total number of samples rejected due to not being out of the expected order.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_total","description":"Total number of checkpoint deletions attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds","description":"The timestamp of the oldest exemplar stored in circular storage. Useful to check for what timerange the current exemplar buffer limit allows. This usually means the last timestampfor all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_page_flushes_total","description":"Total number of page flushes.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":186}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_min_time","description":"Minimum time bound of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601637670}]}},{"name":"prometheus_tsdb_lowest_timestamp","description":"Lowest timestamp value stored in the database. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601637670}]}},{"name":"go_memstats_stack_sys_bytes","description":"Number of bytes obtained from system for stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":2162688}]}},{"name":"prometheus_config_last_reload_successful","description":"Whether the last configuration reload attempt was successful.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1}]}},{"name":"prometheus_engine_queries_concurrent_max","description":"The max number of concurrent queries.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":20}]}},{"name":"prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total","description":"Total number of scrapes that hit the native histogram bucket limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_blocks_loaded","description":"Number of currently loaded data blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_compactions_triggered_total","description":"Total number of triggered compactions for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_symbol_table_size_bytes","description":"Size of symbol table in memory for loaded blocks","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_tombstone_cleanup_seconds","description":"The time taken to recompact blocks to remove tombstones.","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.005,0.01,0.025,0.05,0.1,0.25,0.5,1,2.5,5,10]}],"aggregationTemporality":2}},{"name":"prometheus_engine_query_log_enabled","description":"State of the query log.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_wal_fsync_duration_seconds","description":"Duration of write log fsync.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_compaction_chunk_samples","description":"Final number of samples on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[4,6,9,13.5,20.25,30.375,45.5625,68.34375,102.515625,153.7734375,230.66015625,345.990234375]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_time_retentions_total","description":"The number of times that blocks were deleted because the maximum time limit was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_buck_hash_sys_bytes","description":"Number of bytes used by the profiling bucket hash table.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1483740}]}},{"name":"go_memstats_heap_idle_bytes","description":"Number of heap bytes waiting to be used.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":12566528}]}},{"name":"go_threads","description":"Number of OS threads created.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":37}]}},{"name":"prometheus_sd_consul_rpc_failures_total","description":"The number of Consul RPC call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_exceeded_sample_limit_total","description":"Total number of scrapes that hit the sample limit and were rejected.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_chunk_range_seconds","description":"Final time range of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[100,400,1600,6400,25600,102400,409600,1638400,6553600,26214400]}],"aggregationTemporality":2}},{"name":"prometheus_target_scrape_pools_failed_total","description":"Total number of scrape pool creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compactions_failed_total","description":"Total number of compactions that failed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mspan_inuse_bytes","description":"Number of bytes in use by mspan structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":649760}]}},{"name":"prometheus_api_remote_write_invalid_labels_samples_total","description":"The total number of remote write samples which contains invalid labels.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_queue_length","description":"The number of alert notifications in the queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_rule_evaluation_duration_seconds","description":"The duration for a rule to execute.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_file_read_errors_total","description":"The number of File-SD read errors.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_scan_duration_seconds","description":"The duration of the File-SD scan in seconds.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"go_memstats_mspan_sys_bytes","description":"Number of bytes used for mspan structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":685440}]}},{"name":"prometheus_tsdb_compactions_total","description":"Total number of compactions that were executed for the partition.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_exemplar_exemplars_appended_total","description":"Total number of appended exemplars.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0.005617117}]}},{"name":"prometheus_tsdb_reloads_total","description":"Number of times the database reloaded block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":3}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_gc_sys_bytes","description":"Number of bytes used for garbage collection system metadata.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":8953168}]}},{"name":"go_memstats_other_sys_bytes","description":"Number of bytes used for other system allocations.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":5998476}]}},{"name":"net_conntrack_dialer_conn_attempted_total","description":"Total number of connections attempted by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_dropped_total","description":"Total number of alerts dropped due to errors when sending to Alertmanager.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_cache_flush_forced_total","description":"How many times a scrape cache was flushed due to getting big while scrapes are failing.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_out_of_bounds_total","description":"Total number of samples rejected due to timestamp falling outside of the time bounds.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_vertical_compactions_total","description":"Total number of compactions done on overlapping blocks.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_errors_total","description":"Total number of errors that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_file_watcher_errors_total","description":"The number of File-SD errors caused by filesystem watch failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_linode_failures_total","description":"Number of Linode service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_failed_total","description":"Total number of failed scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_template_text_expansion_failures_total","description":"The total number of template text expansion failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_compaction_duration_seconds","description":"Duration of compaction runs","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[1,2,4,8,16,32,64,128,256,512,1024,2048,4096,8192]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_truncations_total","description":"Total number of head truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_open_fds","description":"Number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":14}]}},{"name":"prometheus_tsdb_exemplar_exemplars_in_storage","description":"Number of exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_series_with_exemplars_in_storage","description":"Number of series with exemplars currently in circular storage.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_too_old_samples_total","description":"Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of time window.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_objects","description":"Number of allocated objects.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":155274}]}},{"name":"prometheus_api_remote_read_queries","description":"The current number of remote read queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_sd_dns_lookups_total","description":"The number of DNS-SD lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_nomad_failures_total","description":"Number of nomad service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_sys_bytes","description":"Number of heap bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":43974656}]}},{"name":"prometheus_remote_storage_exemplars_in_total","description":"Exemplars in to remote storage, compare to exemplars out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_failed_total","description":"Total number of checkpoint creations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_not_found_total","description":"Total number of requests for series that were not found.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_order_samples_total","description":"Total number of out of order samples ingestion failed attempts due to out of order being disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_wal_writes_failed_total","description":"Total number of write log writes that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_web_federation_warnings_total","description":"Total number of warnings that occurred while sending federation responses.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_azure_failures_total","description":"Number of Azure service discovery refresh failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_dns_lookup_failures_total","description":"The number of DNS-SD lookup failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_info","description":"Information about the Go environment.","gauge":{"dataPoints":[{"attributes":[{"key":"version","value":{"stringValue":"go1.20.6"}}],"timeUnixNano":"1700601811616000000","asDouble":1}]}},{"name":"go_memstats_alloc_bytes_total","description":"Total number of bytes allocated, even if freed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":230763240}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_inuse_bytes","description":"Number of heap bytes that are in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":31408128}]}},{"name":"go_memstats_mcache_inuse_bytes","description":"Number of bytes in use by mcache structures.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":38400}]}},{"name":"go_memstats_sys_bytes","description":"Number of bytes obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":63304968}]}},{"name":"prometheus_remote_storage_histograms_in_total","description":"HistogramSamples in to remote storage, compare to histograms out for queue managers.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_duration_seconds","description":"The duration of a Kuma MADS fetch call.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_tsdb_head_truncations_failed_total","description":"Total number of head truncations that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_http_request_duration_seconds","description":"Histogram of latencies for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","count":"170","sum":0.2031089129999999,"bucketCounts":["170","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601811616000000","count":"33","sum":0.076513399,"bucketCounts":["33","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0.1,0.2,0.4,1,3,8,20,60,120]}],"aggregationTemporality":2}},{"name":"prometheus_sd_kuma_fetch_failures_total","description":"The number of Kuma MADS fetch call failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_creations_total","description":"Total number of checkpoint creations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_mmap_chunk_corruptions_total","description":"Total number of memory-mapped chunk corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_reloads_failures_total","description":"Number of times the database failed to reloadBlocks block data from disk.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1}]}},{"name":"prometheus_tsdb_wal_truncations_total","description":"Total number of write log truncations attempted.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_heap_alloc_bytes","description":"Number of heap bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":27006792}]}},{"name":"net_conntrack_dialer_conn_closed_total","description":"Total number of connections closed which originated from the dialer of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_max_fds","description":"Maximum number of open file descriptors.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1048576}]}},{"name":"process_virtual_memory_bytes","description":"Virtual memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":825593856}]}},{"name":"prometheus_treecache_watcher_goroutines","description":"The current number of watcher goroutines.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_retention_limit_bytes","description":"Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"go_memstats_stack_inuse_bytes","description":"Number of bytes in use by the stack allocator.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":2162688}]}},{"name":"process_start_time_seconds","description":"Start time of the process since unix epoch in seconds.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601636.42}]}},{"name":"prometheus_http_response_size_bytes","description":"Histogram of response size for HTTP requests.","histogram":{"dataPoints":[{"attributes":[{"key":"handler","value":{"stringValue":"/api/v1/write"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","count":"170","sum":0,"bucketCounts":["170","0","0","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]},{"attributes":[{"key":"handler","value":{"stringValue":"/metrics"}}],"startTimeUnixNano":"1700601651616000000","timeUnixNano":"1700601811616000000","count":"33","sum":247786,"bucketCounts":["0","0","33","0","0","0","0","0","0"],"explicitBounds":[100,1000,10000,100000,1000000,10000000,100000000,1000000000]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_wal_corruptions_total","description":"Total number of WAL corruptions.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_dialer_conn_established_total","description":"Total number of connections successfully established by the given dialer a given name.","sum":{"dataPoints":[{"attributes":[{"key":"dialer_name","value":{"stringValue":"default"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_checkpoint_deletions_failed_total","description":"Total number of checkpoint deletions that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_max_time","description":"Maximum timestamp of the head block. The unit is decided by the library consumer.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601810439}]}},{"name":"prometheus_tsdb_head_samples_appended_total","description":"Total number of appended samples.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":41479},{"attributes":[{"key":"type","value":{"stringValue":"histogram"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":418}]}},{"name":"go_memstats_heap_released_bytes","description":"Number of heap bytes released to OS.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":7307264}]}},{"name":"prometheus_sd_consul_rpc_duration_seconds","description":"The duration of a Consul RPC call in seconds.","summary":{"dataPoints":[{"attributes":[{"key":"call","value":{"stringValue":"service"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]},{"attributes":[{"key":"call","value":{"stringValue":"services"}},{"key":"endpoint","value":{"stringValue":"catalog"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","quantileValues":[{"quantile":0.5,"value":"NaN"},{"quantile":0.9,"value":"NaN"},{"quantile":0.99,"value":"NaN"}]}]}},{"name":"prometheus_sd_kubernetes_events_total","description":"The number of Kubernetes events handled.","sum":{"dataPoints":[{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"add"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"delete"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpoints"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"endpointslice"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"ingress"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"node"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"pod"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"event","value":{"stringValue":"update"}},{"key":"role","value":{"stringValue":"service"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_sd_kuma_fetch_skipped_updates_total","description":"The number of Kuma MADS fetch calls that result in no updates to the targets.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_exceeded_target_limit_total","description":"Total number of times scrape pools hit the target limit, during sync or config reload.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pools_total","description":"Total number of scrape pool creation attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_queries","description":"The current number of queries being executed or waiting.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_remote_storage_highest_timestamp_in_seconds","description":"Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601810}]}},{"name":"prometheus_tsdb_wal_storage_size_bytes","description":"Size of the write log directory.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":253677}]}},{"name":"go_memstats_mcache_sys_bytes","description":"Number of bytes used for mcache structures obtained from system.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":46800}]}},{"name":"process_virtual_memory_max_bytes","description":"Maximum amount of virtual memory available in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":18446744073709552000}]}},{"name":"prometheus_remote_storage_string_interner_zero_reference_releases_total","description":"The number of times release has been called for strings that are not interned.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_data_replay_duration_seconds","description":"Time taken to replay the data on disk.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0.003792276}]}},{"name":"prometheus_tsdb_size_retentions_total","description":"The number of times that blocks were deleted because the maximum number of bytes was exceeded.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_last_gc_time_seconds","description":"Number of seconds since 1970 of last garbage collection.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1700601798.2886453}]}},{"name":"net_conntrack_listener_conn_accepted_total","description":"Total number of connections opened to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":2}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_resident_memory_bytes","description":"Resident memory size in bytes.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":86392832}]}},{"name":"prometheus_target_scrapes_exceeded_body_size_limit_total","description":"Total number of scrapes that hit the body size limit","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_active_appenders","description":"Number of currently active appender transactions","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_head_chunks_removed_total","description":"Total number of chunks removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_log_failures_total","description":"The number of query log failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_engine_query_samples_total","description":"The total number of samples loaded by all queries.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_treecache_zookeeper_failures_total","description":"The total number of ZooKeeper failures.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_gc_duration_seconds","description":"Runtime of garbage collection in the head block.","summary":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000"}]}},{"name":"prometheus_tsdb_storage_blocks_bytes","description":"The number of bytes that are currently used for local storage by all blocks.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":418}]}},{"name":"prometheus_tsdb_wal_completed_pages_total","description":"Total number of completed pages.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":7}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_lookups_total","description":"Total number of pointer lookups.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_build_info","description":"A metric with a constant '1' value labeled by version, revision, branch, goversion from which prometheus was built, and the goos and goarch for the build.","gauge":{"dataPoints":[{"attributes":[{"key":"branch","value":{"stringValue":"HEAD"}},{"key":"goarch","value":{"stringValue":"amd64"}},{"key":"goos","value":{"stringValue":"linux"}},{"key":"goversion","value":{"stringValue":"go1.20.6"}},{"key":"revision","value":{"stringValue":"cbb69e51423565ec40f46e74f4ff2dbb3b7fb4f0"}},{"key":"tags","value":{"stringValue":"netgo,builtinassets,stringlabels"}},{"key":"version","value":{"stringValue":"2.46.0"}}],"timeUnixNano":"1700601811616000000","asDouble":1}]}},{"name":"prometheus_notifications_queue_capacity","description":"The capacity of the alert notifications queue.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":10000}]}},{"name":"prometheus_tsdb_compaction_chunk_size_bytes","description":"Final size of chunks on their first compaction","histogram":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","sum":0,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[32,48,72,108,162,243,364.5,546.75,820.125,1230.1875,1845.28125,2767.921875]}],"aggregationTemporality":2}},{"name":"prometheus_tsdb_head_chunks_created_total","description":"Total number of chunks created in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":1360}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_out_of_bound_samples_total","description":"Total number of out of bound samples ingestion failed attempts with out of order support disabled.","sum":{"dataPoints":[{"attributes":[{"key":"type","value":{"stringValue":"float"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"net_conntrack_listener_conn_closed_total","description":"Total number of connections closed that were made to the listener of a given name.","sum":{"dataPoints":[{"attributes":[{"key":"listener_name","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrape_pool_reloads_total","description":"Total number of scrape pool reloads.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_snapshot_replay_error_total","description":"Total number snapshot replays that failed.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_notifications_alertmanagers_discovered","description":"The number of alertmanagers discovered and active.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"prometheus_tsdb_exemplar_out_of_order_exemplars_total","description":"Total number of out of order exemplar ingestion failed attempts.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_head_series_removed_total","description":"Total number of series removed in the head","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_isolation_low_watermark","description":"The lowest TSDB append ID that is still referenced.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":170}]}},{"name":"prometheus_tsdb_wal_segment_current","description":"Write log segment index that TSDB is currently writing to.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":0}]}},{"name":"promhttp_metric_handler_requests_total","description":"Total number of scrapes by HTTP status code.","sum":{"dataPoints":[{"attributes":[{"key":"code","value":{"stringValue":"200"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":33},{"attributes":[{"key":"code","value":{"stringValue":"500"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0},{"attributes":[{"key":"code","value":{"stringValue":"503"}}],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_alloc_bytes","description":"Number of bytes allocated and still in use.","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":27006792}]}},{"name":"go_memstats_frees_total","description":"Total number of frees.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":1409873}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"go_memstats_mallocs_total","description":"Total number of mallocs.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":1565147}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_target_scrapes_sample_duplicate_timestamp_total","description":"Total number of samples rejected due to duplicate timestamps but different values.","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646616000000","timeUnixNano":"1700601811616000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"prometheus_tsdb_clean_start","description":"-1: lockfile is disabled. 0: a lockfile from a previous execution was replaced. 1: lockfile creation was clean","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811616000000","asDouble":1}]}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"otel-collector"}},{"key":"service.instance.id","value":{"stringValue":"localhost:8888"}},{"key":"net.host.port","value":{"stringValue":"8888"}},{"key":"http.scheme","value":{"stringValue":"http"}}]},"scopeMetrics":[{"scope":{"name":"otelcol/prometheusreceiver","version":"0.89.0"},"metrics":[{"name":"otelcol_rpc_client_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"274","sum":1431051033,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","274"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"171","sum":90222246,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","171"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"274","sum":274,"bucketCounts":["0","274","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"171","sum":0,"bucketCounts":["171","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"37","sum":37,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"169","sum":169,"bucketCounts":["0","169","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_process_uptime_total","description":"Uptime of the process","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":343.76409833}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_accepted_spans_total","description":"Number of spans successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":5562}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_client_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"274","sum":548,"bucketCounts":["0","274","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_client_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"274","sum":274,"bucketCounts":["0","274","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"12"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"171","sum":80,"bucketCounts":["91","80","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_request_size","description":"Measures size of RPC request messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"37","sum":484367,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","34"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"169","sum":1995892,"bucketCounts":["0","0","0","0","0","0","0","0","31","0","2","24","11","0","0","101"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"scrape_series_added","description":"The approximate number of new series in this scrape","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":398}]}},{"name":"otelcol_processor_batch_timeout_trigger_send_total","description":"Number of times the batch was sent due to a timeout trigger","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":308}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_memory_rss","description":"Total physical memory (resident set size)","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":209453056}]}},{"name":"otelcol_processor_batch_metadata_cardinality","description":"Number of distinct metadata value combinations being processed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":6}]}},{"name":"otelcol_receiver_accepted_metric_points_total","description":"Number of metric points successfully pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":1748},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":10958}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_sent_metric_points_total","description":"Number of metric points successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":12706},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":12706}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_send_failed_metric_points_total","description":"Number of metric points in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":12706},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_receiver_refused_metric_points_total","description":"Number of metric points that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":0},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_exporter_queue_capacity","description":"Fixed capacity of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601811873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601811873000000","asDouble":1000},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601811873000000","asDouble":10000}]}},{"name":"otelcol_exporter_sent_spans_total","description":"Number of spans successfully sent to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":5562},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":5562}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":140501272}]}},{"name":"otelcol_rpc_client_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"80","sum":228828,"bucketCounts":["0","0","0","0","0","0","0","0","0","66","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"274","sum":3997900,"bucketCounts":["0","0","0","0","0","0","0","0","62","0","4","0","6","0","0","202"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"up","description":"The scraping was successful","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":1}]}},{"name":"scrape_samples_scraped","description":"The number of samples the target exposed","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":398}]}},{"name":"scrape_samples_post_metric_relabeling","description":"The number of samples remaining after metric relabeling was applied","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":398}]}},{"name":"otelcol_exporter_send_failed_spans_total","description":"Number of spans in failed attempts to send to destination.","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_processor_batch_batch_send_size","description":"Number of units in the batch","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"308","sum":18268,"bucketCounts":["103","4","67","93","6","1","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000]}],"aggregationTemporality":2}},{"name":"otelcol_receiver_refused_spans_total","description":"Number of spans that could not be pushed into the pipeline.","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":0}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_cpu_seconds_total","description":"Total CPU user and system time in seconds","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":3.85}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":61773896}]}},{"name":"otelcol_process_runtime_total_alloc_bytes_total","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","sum":{"dataPoints":[{"attributes":[],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","asDouble":821082712}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"otelcol_rpc_server_duration","description":"Measures the duration of inbound RPC.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"37","sum":8548309,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","37"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"169","sum":28107721,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","169"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_target_info","description":"Target metadata","gauge":{"dataPoints":[{"attributes":[{"key":"service_instance_id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service_name","value":{"stringValue":"otelcol-contrib"}},{"key":"service_version","value":{"stringValue":"0.89.0"}}],"timeUnixNano":"1700601811873000000","asDouble":1}]}},{"name":"scrape_duration_seconds","description":"Duration of the scrape","unit":"s","gauge":{"dataPoints":[{"attributes":[],"timeUnixNano":"1700601811873000000","asDouble":0.00331491}]}},{"name":"otelcol_exporter_queue_size","description":"Current size of the retry queue (in batches)","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601811873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601811873000000","asDouble":0},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601811873000000","asDouble":0}]}},{"name":"otelcol_rpc_server_responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"37","sum":37,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_grpc_status_code","value":{"stringValue":"0"}},{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"169","sum":169,"bucketCounts":["0","169","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}},{"name":"otelcol_rpc_server_response_size","description":"Measures size of RPC response messages (uncompressed).","histogram":{"dataPoints":[{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"37","sum":74,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]},{"attributes":[{"key":"rpc_method","value":{"stringValue":"Export"}},{"key":"rpc_service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc_system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601646873000000","timeUnixNano":"1700601811873000000","count":"169","sum":338,"bucketCounts":["0","169","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000]}],"aggregationTemporality":2}}]}]}]} +{"resourceMetrics":[{"resource":{"attributes":[{"key":"service.instance.id","value":{"stringValue":"79eb65b8-4d28-4129-abf4-c387dd62f570"}},{"key":"service.name","value":{"stringValue":"otelcol-contrib"}},{"key":"service.version","value":{"stringValue":"0.89.0"}}]},"scopeMetrics":[{"scope":{"name":"go.opentelemetry.io/collector/service/process_telemetry"},"metrics":[{"name":"process_uptime","description":"Uptime of the process","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137479326","timeUnixNano":"1700601813109567026","asDouble":348.76376803000005}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_heap_alloc_bytes","description":"Bytes of allocated heap objects (see 'go doc runtime.MemStats.HeapAlloc')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601813109377828","asInt":"65554744"}]}},{"name":"process_runtime_total_alloc_bytes","description":"Cumulative bytes allocated for heap objects (see 'go doc runtime.MemStats.TotalAlloc')","unit":"By","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137499256","timeUnixNano":"1700601813109571816","asInt":"834079872"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_runtime_total_sys_memory_bytes","description":"Total bytes of memory obtained from the OS (see 'go doc runtime.MemStats.Sys')","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601813109382108","asInt":"140501272"}]}},{"name":"process_cpu_seconds","description":"Total CPU user and system time in seconds","unit":"s","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638137508166","timeUnixNano":"1700601813109573195","asDouble":3.94}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"process_memory_rss","description":"Total physical memory (resident set size)","unit":"By","gauge":{"dataPoints":[{"timeUnixNano":"1700601813109545946","asInt":"209453056"}]}}]},{"scope":{"name":"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc","version":"0.46.0"},"metrics":[{"name":"rpc.client.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601813109578955","count":"172","sum":90543721,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","172"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":259146,"max":2116189},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138214736","timeUnixNano":"1700601813109578955","count":"278","sum":1460575865,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","278"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":572521,"max":19501993}],"aggregationTemporality":2}},{"name":"rpc.client.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138220356","timeUnixNano":"1700601813109581585","count":"278","sum":556,"bucketCounts":["0","278","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.client.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601813109583175","count":"278","sum":4083576,"bucketCounts":["0","0","0","0","0","0","0","0","62","0","4","0","6","0","0","206"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":381,"max":33605},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138225536","timeUnixNano":"1700601813109583175","count":"80","sum":228828,"bucketCounts":["0","0","0","0","0","0","0","0","0","66","0","0","0","1","0","13"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":640,"max":13954}],"aggregationTemporality":2}},{"name":"rpc.client.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601813109584475","count":"172","sum":0,"bucketCounts":["172","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":0},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138232955","timeUnixNano":"1700601813109584475","count":"278","sum":278,"bucketCounts":["0","278","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.client.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"12"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601813109585325","count":"172","sum":80,"bucketCounts":["92","80","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":0,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138237245","timeUnixNano":"1700601813109585325","count":"278","sum":278,"bucketCounts":["0","278","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.duration","description":"Measures the duration of inbound RPC.","unit":"ms","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601813109586455","count":"37","sum":8548309,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","37"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":109368,"max":2029081},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138628890","timeUnixNano":"1700601813109586455","count":"172","sum":28577925,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","172"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":46759,"max":1750565}],"aggregationTemporality":2}},{"name":"rpc.server.request.size","description":"Measures size of RPC request messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601813109588445","count":"37","sum":484367,"bucketCounts":["0","0","0","0","0","0","0","0","0","0","0","1","0","2","0","34"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1931,"max":13900},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138635080","timeUnixNano":"1700601813109588445","count":"172","sum":2038716,"bucketCounts":["0","0","0","0","0","0","0","0","31","0","2","25","11","0","0","103"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":379,"max":31061}],"aggregationTemporality":2}},{"name":"rpc.server.response.size","description":"Measures size of RPC response messages (uncompressed).","unit":"By","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601813109589865","count":"37","sum":74,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2},{"attributes":[{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138641480","timeUnixNano":"1700601813109589865","count":"172","sum":344,"bucketCounts":["0","172","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":2,"max":2}],"aggregationTemporality":2}},{"name":"rpc.server.requests_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601813109590865","count":"37","sum":37,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138648420","timeUnixNano":"1700601813109590865","count":"172","sum":172,"bucketCounts":["0","172","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}},{"name":"rpc.server.responses_per_rpc","description":"Measures the number of messages received per RPC. Should be 1 for all non-streaming RPCs.","unit":"{count}","histogram":{"dataPoints":[{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.trace.v1.TraceService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601813109591685","count":"172","sum":172,"bucketCounts":["0","172","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1},{"attributes":[{"key":"rpc.grpc.status_code","value":{"intValue":"0"}},{"key":"rpc.method","value":{"stringValue":"Export"}},{"key":"rpc.service","value":{"stringValue":"opentelemetry.proto.collector.metrics.v1.MetricsService"}},{"key":"rpc.system","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638138652930","timeUnixNano":"1700601813109591685","count":"37","sum":37,"bucketCounts":["0","37","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[0,5,10,25,50,75,100,250,500,750,1000,2500,5000,7500,10000],"min":1,"max":1}],"aggregationTemporality":2}}],"schemaUrl":"https://opentelemetry.io/schemas/1.17.0"},{"scope":{"name":"go.opentelemetry.io/collector/exporterhelper"},"metrics":[{"name":"exporter/queue_size","description":"Current size of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601813109550506","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601813109562696","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601813109558226","asInt":"0"}]}},{"name":"exporter/queue_capacity","description":"Fixed capacity of the retry queue (in batches)","unit":"1","gauge":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"timeUnixNano":"1700601813109553696","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"timeUnixNano":"1700601813109563336","asInt":"1000"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"timeUnixNano":"1700601813109559206","asInt":"10000"}]}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/exporter"},"metrics":[{"name":"exporter/sent_spans","description":"Number of spans successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601813109596945","asInt":"5665"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109121009","timeUnixNano":"1700601813109596945","asInt":"5665"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_spans","description":"Number of spans in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"otlp/jaeger"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601813109597845","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109130709","timeUnixNano":"1700601813109597845","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/sent_metric_points","description":"Number of metric points successfully sent to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601813109599155","asInt":"12762"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601813109599155","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109137929","timeUnixNano":"1700601813109599155","asInt":"12762"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"exporter/send_failed_metric_points","description":"Number of metric points in failed attempts to send to destination.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"exporter","value":{"stringValue":"prometheusremotewrite"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601813109599835","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"file"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601813109599835","asInt":"0"},{"attributes":[{"key":"exporter","value":{"stringValue":"otlp"}}],"startTimeUnixNano":"1700601638109141439","timeUnixNano":"1700601813109599835","asInt":"12762"}],"aggregationTemporality":2,"isMonotonic":true}}]},{"scope":{"name":"go.opentelemetry.io/collector/processor/batchprocessor"},"metrics":[{"name":"processor/batch/timeout_trigger_send","description":"Number of times the batch was sent due to a timeout trigger","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109359076","timeUnixNano":"1700601813109602485","asInt":"311"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"processor/batch/batch_send_size","description":"Number of units in the batch","unit":"1","histogram":{"dataPoints":[{"attributes":[{"key":"processor","value":{"stringValue":"batch"}}],"startTimeUnixNano":"1700601638109369586","timeUnixNano":"1700601813109602915","count":"311","sum":18427,"bucketCounts":["103","4","68","95","6","1","34","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],"explicitBounds":[10,25,50,75,100,250,500,750,1000,2000,3000,4000,5000,6000,7000,8000,9000,10000,20000,30000,50000,100000],"min":1,"max":258}],"aggregationTemporality":2}},{"name":"processor/batch/metadata_cardinality","description":"Number of distinct metadata value combinations being processed","unit":"1","sum":{"dataPoints":[{"startTimeUnixNano":"1700601638109382696","timeUnixNano":"1700601813109605665","asInt":"6"}],"aggregationTemporality":2}}]},{"scope":{"name":"go.opentelemetry.io/collector/obsreport/receiver"},"metrics":[{"name":"receiver/accepted_spans","description":"Number of spans successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109453875","timeUnixNano":"1700601813109606455","asInt":"5665"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_spans","description":"Number of spans that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109457785","timeUnixNano":"1700601813109607345","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/accepted_metric_points","description":"Number of metric points successfully pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601813109608025","asInt":"1748"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109461495","timeUnixNano":"1700601813109608025","asInt":"11014"}],"aggregationTemporality":2,"isMonotonic":true}},{"name":"receiver/refused_metric_points","description":"Number of metric points that could not be pushed into the pipeline.","unit":"1","sum":{"dataPoints":[{"attributes":[{"key":"receiver","value":{"stringValue":"otlp"}},{"key":"transport","value":{"stringValue":"grpc"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601813109608585","asInt":"0"},{"attributes":[{"key":"receiver","value":{"stringValue":"prometheus"}},{"key":"transport","value":{"stringValue":"http"}}],"startTimeUnixNano":"1700601638109465244","timeUnixNano":"1700601813109608585","asInt":"0"}],"aggregationTemporality":2,"isMonotonic":true}}]}],"schemaUrl":"https://opentelemetry.io/schemas/1.18.0"}]} From 36c4463c7328089f139a205b8fe55ea12e222afb Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 00:26:04 +0300 Subject: [PATCH 024/112] fix(metricsharding): handle empty value type --- internal/metricsharding/consumer.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/internal/metricsharding/consumer.go b/internal/metricsharding/consumer.go index 5a41e923..e5e5621d 100644 --- a/internal/metricsharding/consumer.go +++ b/internal/metricsharding/consumer.go @@ -198,6 +198,9 @@ func (c *Consumer) mapMetrics(ctx context.Context, metrics pmetric.Metrics) (bat var val float64 switch typ := point.ValueType(); typ { + case pmetric.NumberDataPointValueTypeEmpty: + // Just ignore it. + continue case pmetric.NumberDataPointValueTypeInt: // TODO(tdakkota): check for overflow val = float64(point.IntValue()) From a71d93fbf5a61394eca87ed2e77fa87e1b2afb00 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 13:50:08 +0300 Subject: [PATCH 025/112] fix(chstorage): series query building --- internal/chstorage/querier_metrics.go | 23 ++++++++++------------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 060eae0b..59fb8117 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -217,14 +217,12 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin var query strings.Builder fmt.Fprintf(&query, "SELECT * FROM %#[1]q WHERE true\n", table) if !start.IsZero() { - fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(ts) >= %d\n", start.UnixNano()) + fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(timestamp) >= %d\n", start.UnixNano()) } if !end.IsZero() { - fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(ts) <= %d\n", end.UnixNano()) + fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(timestamp) <= %d\n", end.UnixNano()) } for _, m := range matchers { - query.WriteString("\t(") - switch m.Type { case labels.MatchEqual, labels.MatchRegexp: query.WriteString("AND ") @@ -244,10 +242,10 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin fmt.Sprintf("JSONExtractString(resource, %s)", singleQuoted(m.Name)), } } - query.WriteString("\t\t(") + query.WriteString("(\n") for i, sel := range selectors { if i != 0 { - query.WriteString("\t\tOR") + query.WriteString("\tOR ") } // Note: predicate negated above. switch m.Type { @@ -259,12 +257,11 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin return nil, errors.Errorf("unexpected type %q", m.Type) } } - query.WriteString("\t)") + query.WriteString(")") } - - query.WriteString(")\n") + query.WriteString("\n") } - query.WriteString("ORDER BY ts") + query.WriteString("ORDER BY timestamp") return p.doQuery(ctx, query.String()) } @@ -283,9 +280,9 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series Body: query, Result: c.Result(), OnResult: func(ctx context.Context, block proto.Block) error { - for i := 0; i < c.ts.Rows(); i++ { + for i := 0; i < c.timestamp.Rows(); i++ { value := c.value.Row(i) - ts := c.ts.Row(i) + timestamp := c.timestamp.Row(i) attributes := c.attributes.Row(i) resource := c.resource.Row(i) @@ -303,7 +300,7 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series } s.series.values = append(s.series.values, value) - s.series.ts = append(s.series.ts, ts.UnixMilli()) + s.series.ts = append(s.series.ts, timestamp.UnixMilli()) if err := parseLabels(resource, s.labels); err != nil { return errors.Wrap(err, "parse resource") } From e8a35ebbe5bc4b74a79fdf5bddd7877f8bd61022 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 13:52:49 +0300 Subject: [PATCH 026/112] fix(chstorage): add name as label to series result --- internal/chstorage/querier_metrics.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 59fb8117..843a5788 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -181,6 +181,7 @@ func (p *promQuerier) Select(ctx context.Context, _ bool, hints *storage.SelectH } type seriesKey struct { + name string attributes string resource string } @@ -281,12 +282,14 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series Result: c.Result(), OnResult: func(ctx context.Context, block proto.Block) error { for i := 0; i < c.timestamp.Rows(); i++ { + name := c.name.Row(i) value := c.value.Row(i) timestamp := c.timestamp.Row(i) attributes := c.attributes.Row(i) resource := c.resource.Row(i) key := seriesKey{ + name: name, attributes: attributes, resource: resource, } @@ -301,6 +304,8 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series s.series.values = append(s.series.values, value) s.series.ts = append(s.series.ts, timestamp.UnixMilli()) + + s.labels["__name__"] = name if err := parseLabels(resource, s.labels); err != nil { return errors.Wrap(err, "parse resource") } From aa4535778f9653a7e4ce38e4f9bd2f8dd2326229 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 13:53:14 +0300 Subject: [PATCH 027/112] chore(chstorage): make field name match column name --- internal/chstorage/columns_metrics.go | 10 +++++----- internal/chstorage/inserter_metrics.go | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/internal/chstorage/columns_metrics.go b/internal/chstorage/columns_metrics.go index 65dee3ca..f34f4a78 100644 --- a/internal/chstorage/columns_metrics.go +++ b/internal/chstorage/columns_metrics.go @@ -4,7 +4,7 @@ import "github.com/ClickHouse/ch-go/proto" type metricColumns struct { name *proto.ColLowCardinality[string] - ts *proto.ColDateTime64 + timestamp *proto.ColDateTime64 value proto.ColFloat64 attributes proto.ColStr resource proto.ColStr @@ -12,8 +12,8 @@ type metricColumns struct { func newMetricColumns() *metricColumns { return &metricColumns{ - name: new(proto.ColStr).LowCardinality(), - ts: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + name: new(proto.ColStr).LowCardinality(), + timestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), } } @@ -30,7 +30,7 @@ func (c *metricColumns) StaticColumns() []string { func (c *metricColumns) Input() proto.Input { input := proto.Input{ {Name: "name", Data: c.name}, - {Name: "timestamp", Data: c.ts}, + {Name: "timestamp", Data: c.timestamp}, {Name: "value", Data: c.value}, {Name: "attributes", Data: c.attributes}, {Name: "resource", Data: c.resource}, @@ -41,7 +41,7 @@ func (c *metricColumns) Input() proto.Input { func (c *metricColumns) Result() proto.Results { return proto.Results{ {Name: "name", Data: c.name}, - {Name: "timestamp", Data: c.ts}, + {Name: "timestamp", Data: c.timestamp}, {Name: "value", Data: &c.value}, {Name: "attributes", Data: &c.attributes}, {Name: "resource", Data: &c.resource}, diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index e8d6303b..cb7ca86f 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -93,7 +93,7 @@ func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics, collect collectLabels(attrs) c.name.Append(name) - c.ts.Append(ts) + c.timestamp.Append(ts) c.value.Append(val) c.attributes.Append(encodeAttributes(attrs)) c.resource.Append(encodeAttributes(res)) From 9e1b2e756c8dc1baf2e710834ebcd1ae86604573 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 13:53:24 +0300 Subject: [PATCH 028/112] test(prome2e): add series query test --- integration/prome2e/common_test.go | 52 ++++++++++++++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/integration/prome2e/common_test.go b/integration/prome2e/common_test.go index 564e8778..2438b728 100644 --- a/integration/prome2e/common_test.go +++ b/integration/prome2e/common_test.go @@ -99,4 +99,56 @@ func runTest( } } }) + t.Run("Series", func(t *testing.T) { + t.Run("ByName", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1600000000.0`), + End: promapi.NewOptPrometheusTimestamp(`1800000000.0`), + Match: []string{ + `prometheus_http_requests_total{}`, + }, + }) + a.NoError(err) + + a.NotEmpty(r.Data) + for _, labels := range r.Data { + a.Equal("prometheus_http_requests_total", labels["__name__"]) + } + }) + t.Run("Matchers", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1600000000.0`), + End: promapi.NewOptPrometheusTimestamp(`1800000000.0`), + Match: []string{ + `prometheus_http_requests_total{ + code="200", + handler=~"/api/v1.+", + handler!="/api/v1/series", + handler!~"/api/v1/query(_range)?" + }`, + }, + }) + a.NoError(err) + + a.NotEmpty(r.Data) + for _, labels := range r.Data { + a.Equal("200", labels["code"]) + + handler := labels["handler"] + // Check that handler=~"/api/v1.+" is satisfied. + a.Contains(handler, "/api/v1") + + // Check that handler!="/api/v1/series" is satisfied. + a.NotEqual("/api/v1/series", handler) + + // Check that handler!~"/api/v1/query(_range)?" is satisfied. + a.NotEqual("/api/v1/query", handler) + a.NotEqual("/api/v1/query_range", handler) + } + }) + }) } From 92739a77dca18ce78f24770e0c292b39b6c861c6 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 16:58:26 +0300 Subject: [PATCH 029/112] fix(chstorage): use constructor for series set --- internal/chstorage/querier_metrics.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 843a5788..b3c4e34b 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -333,9 +333,7 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series result = append(result, s.series) } - return &seriesSet{ - set: result, - }, nil + return newSeriesSet(result), nil } type seriesSet struct { From 640a63419b77425279b99f33a8c068908e327f22 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 17:00:51 +0300 Subject: [PATCH 030/112] test(prome2e): add more series endpoint tests --- integration/prome2e/common_test.go | 83 +++++++++++++++++++++++++++++- 1 file changed, 82 insertions(+), 1 deletion(-) diff --git a/integration/prome2e/common_test.go b/integration/prome2e/common_test.go index 2438b728..fd021467 100644 --- a/integration/prome2e/common_test.go +++ b/integration/prome2e/common_test.go @@ -117,7 +117,7 @@ func runTest( a.Equal("prometheus_http_requests_total", labels["__name__"]) } }) - t.Run("Matchers", func(t *testing.T) { + t.Run("OneMatcher", func(t *testing.T) { a := require.New(t) r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ @@ -150,5 +150,86 @@ func runTest( a.NotEqual("/api/v1/query_range", handler) } }) + t.Run("MultipleMatchers", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1600000000.0`), + End: promapi.NewOptPrometheusTimestamp(`1800000000.0`), + Match: []string{ + `prometheus_http_requests_total{ + handler="/api/v1/query" + }`, + `prometheus_http_requests_total{ + handler="/api/v1/series" + }`, + }, + }) + a.NoError(err) + + a.NotEmpty(r.Data) + for _, labels := range r.Data { + a.Contains([]string{ + "/api/v1/query", + "/api/v1/series", + }, labels["handler"]) + } + }) + t.Run("OutOfRange", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1000000000.0`), + End: promapi.NewOptPrometheusTimestamp(`1100000000.0`), + Match: []string{ + `prometheus_http_requests_total{}`, + }, + }) + a.NoError(err) + a.Empty(r.Data) + }) + t.Run("NoMatch", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1600000000.0`), + End: promapi.NewOptPrometheusTimestamp(`1800000000.0`), + Match: []string{ + `prometheus_http_requests_total{ + clearly="not_exist" + }`, + }, + }) + a.NoError(err) + a.Empty(r.Data) + }) + t.Run("InvalidTimestamp", func(t *testing.T) { + a := require.New(t) + + _, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1600000000.0`), + End: promapi.NewOptPrometheusTimestamp(`abcd`), + Match: []string{ + `prometheus_http_requests_total{}`, + }, + }) + perr := new(promapi.FailStatusCode) + a.ErrorAs(err, &perr) + a.Equal(promapi.FailErrorTypeBadData, perr.Response.ErrorType) + }) + t.Run("InvalidMatcher", func(t *testing.T) { + a := require.New(t) + + _, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1600000000.0`), + End: promapi.NewOptPrometheusTimestamp(`1800000000.0`), + Match: []string{ + `\{\}`, + }, + }) + perr := new(promapi.FailStatusCode) + a.ErrorAs(err, &perr) + a.Equal(promapi.FailErrorTypeBadData, perr.Response.ErrorType) + }) }) } From 779d1681b488a21f986874219a868789968f3109 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 22 Nov 2023 23:39:24 +0300 Subject: [PATCH 031/112] fix(otedlb): set timeout for engine --- cmd/oteldb/app.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/cmd/oteldb/app.go b/cmd/oteldb/app.go index 8d555491..313020ac 100644 --- a/cmd/oteldb/app.go +++ b/cmd/oteldb/app.go @@ -195,7 +195,13 @@ func (app *App) trySetupProm() error { return nil } - engine := promql.NewEngine(promql.EngineOpts{}) + engine := promql.NewEngine(promql.EngineOpts{ + // The default timeout is 0, which makes + // all queries to fail with timeout error. + // + // TODO(tdakkota): make configurable. + Timeout: time.Minute, + }) prom := promhandler.NewPromAPI(engine, q, promhandler.PromAPIOptions{}) s, err := promapi.NewServer(prom, From 5864f400d08a6a9e831c4dcc190890510c3ae857 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Thu, 23 Nov 2023 01:37:27 +0300 Subject: [PATCH 032/112] fix(otedlb): set samples limit for engine --- cmd/oteldb/app.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/cmd/oteldb/app.go b/cmd/oteldb/app.go index 313020ac..d9d2a2ad 100644 --- a/cmd/oteldb/app.go +++ b/cmd/oteldb/app.go @@ -196,11 +196,12 @@ func (app *App) trySetupProm() error { } engine := promql.NewEngine(promql.EngineOpts{ - // The default timeout is 0, which makes - // all queries to fail with timeout error. + // These fields are zero by default, which makes + // all queries to fail with error. // // TODO(tdakkota): make configurable. - Timeout: time.Minute, + Timeout: time.Minute, + MaxSamples: 1_000_000, }) prom := promhandler.NewPromAPI(engine, q, promhandler.PromAPIOptions{}) From f3ad6e12065a6aee28963a60b11bedb3f7fc9102 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Thu, 23 Nov 2023 01:38:02 +0300 Subject: [PATCH 033/112] test(prome2e): set engine limits properly --- integration/prome2e/common_test.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/integration/prome2e/common_test.go b/integration/prome2e/common_test.go index fd021467..b8976169 100644 --- a/integration/prome2e/common_test.go +++ b/integration/prome2e/common_test.go @@ -5,6 +5,7 @@ import ( "net/http/httptest" "os" "testing" + "time" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" @@ -40,7 +41,10 @@ func setupDB( } } - engine := promql.NewEngine(promql.EngineOpts{}) + engine := promql.NewEngine(promql.EngineOpts{ + Timeout: time.Minute, + MaxSamples: 1_000_000, + }) api := promhandler.NewPromAPI(engine, querier, promhandler.PromAPIOptions{}) promh, err := promapi.NewServer(api) require.NoError(t, err) From 9a95bfab450ab8ae476638b622112e997ea00e7c Mon Sep 17 00:00:00 2001 From: tdakkota Date: Thu, 23 Nov 2023 01:37:03 +0300 Subject: [PATCH 034/112] fix(chstorage): handle seek to the first point properly --- internal/chstorage/querier_metrics.go | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index b3c4e34b..d69958b5 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -431,12 +431,19 @@ func (p *pointIterator) Next() chunkenc.ValueType { func (p *pointIterator) Seek(seek int64) chunkenc.ValueType { // Find the closest value. idx, _ := slices.BinarySearch(p.ts, seek) - if idx >= len(p.ts) { + switch { + case idx >= len(p.ts): + // Outside of the range. p.n = len(p.ts) return chunkenc.ValNone + case idx < 1: + // Move to the first point. + p.n = 0 + return chunkenc.ValFloat + default: + p.n = idx - 1 + return chunkenc.ValFloat } - p.n = idx - 1 - return chunkenc.ValFloat } // At returns the current timestamp/value pair if the value is a float. From c5ed88708a4669e805c5d4b59e92030a266b3f60 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Thu, 23 Nov 2023 13:53:37 +0300 Subject: [PATCH 035/112] test(prome2e): make a simple PromQL query --- integration/prome2e/common_test.go | 30 ++++++++++++++++++++++++++++++ integration/prome2e/prom_e2e.go | 17 ++++++++++++++++- 2 files changed, 46 insertions(+), 1 deletion(-) diff --git a/integration/prome2e/common_test.go b/integration/prome2e/common_test.go index b8976169..8297df6d 100644 --- a/integration/prome2e/common_test.go +++ b/integration/prome2e/common_test.go @@ -4,12 +4,14 @@ import ( "context" "net/http/httptest" "os" + "strconv" "testing" "time" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/pdata/pcommon" "github.com/go-faster/oteldb/integration/prome2e" "github.com/go-faster/oteldb/internal/otelreceiver" @@ -236,4 +238,32 @@ func runTest( a.Equal(promapi.FailErrorTypeBadData, perr.Response.ErrorType) }) }) + t.Run("QueryRange", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetQueryRange(ctx, promapi.GetQueryRangeParams{ + Query: `count(prometheus_http_requests_total{})`, + Start: getPromTS(set.Start), + End: getPromTS(set.End), + Step: "5s", + }) + a.NoError(err) + + data := r.Data + a.Equal(promapi.MatrixData, data.Type) + + mat := data.Matrix.Result + a.Len(mat, 1) + values := mat[0].Values + a.NotEmpty(values) + + for _, point := range values { + a.Equal(float64(51), point.V) + } + }) +} + +func getPromTS(ts pcommon.Timestamp) promapi.PrometheusTimestamp { + v := strconv.FormatInt(ts.AsTime().Unix(), 10) + return promapi.PrometheusTimestamp(v) } diff --git a/integration/prome2e/prom_e2e.go b/integration/prome2e/prom_e2e.go index b7134422..1dbc1496 100644 --- a/integration/prome2e/prom_e2e.go +++ b/integration/prome2e/prom_e2e.go @@ -13,8 +13,10 @@ import ( // BatchSet is a set of batches. type BatchSet struct { Batches []pmetric.Metrics + Labels map[string]map[string]struct{} - Labels map[string]map[string]struct{} + Start pcommon.Timestamp + End pcommon.Timestamp } // ParseBatchSet parses JSON batches from given reader. @@ -74,13 +76,17 @@ func (s *BatchSet) addMetric(metric pmetric.Metric) error { for i := 0; i < points.Len(); i++ { point := points.At(i) s.addLabels(point.Attributes()) + s.addTimestamp(point.Timestamp()) } + return nil case pmetric.MetricTypeSum: points := metric.Sum().DataPoints() for i := 0; i < points.Len(); i++ { point := points.At(i) s.addLabels(point.Attributes()) + s.addTimestamp(point.Timestamp()) } + return nil case pmetric.MetricTypeHistogram: case pmetric.MetricTypeExponentialHistogram: case pmetric.MetricTypeSummary: @@ -91,6 +97,15 @@ func (s *BatchSet) addMetric(metric pmetric.Metric) error { return nil } +func (s *BatchSet) addTimestamp(ts pcommon.Timestamp) { + if s.Start == 0 || ts < s.Start { + s.Start = ts + } + if ts > s.End { + s.End = ts + } +} + func (s *BatchSet) addLabels(m pcommon.Map) { m.Range(func(k string, v pcommon.Value) bool { switch t := v.Type(); t { From a44da37ff817af3fc4184713f7135d04107dddb6 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Fri, 24 Nov 2023 17:20:34 +0300 Subject: [PATCH 036/112] feat(dev.local.ch): more compatibility testing --- dev/local/ch/docker-compose.yml | 37 +++++++++++-- dev/local/grafana/datasources.yaml | 55 ++++++++++++++++-- dev/local/otelcol.yml | 22 ++++++-- dev/local/qryncol.yml | 89 ++++++++++++++++++++++++++++++ 4 files changed, 190 insertions(+), 13 deletions(-) create mode 100644 dev/local/qryncol.yml diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index e066fa66..c577d1bf 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -1,7 +1,7 @@ version: "3" volumes: - prom_data: + prom: services: clickhouse: @@ -12,6 +12,33 @@ services: - "9000:9000" volumes: - ./clickhouse.xml:/etc/clickhouse-server/config.d/monitoring.xml + healthcheck: + test: ['CMD', 'wget', '--spider', '-q', '127.0.0.1:8123/ping'] + interval: 1s + timeout: 1s + retries: 30 + + # Add https://github.com/metrico/qryn for compatibility testing. + qryn: + image: qxip/qryn:latest + ports: + - "3101:3100" + restart: on-failure + environment: + - CLICKHOUSE_SERVER=clickhouse + - OTEL_EXPORTER_OTLP_ENDPOINT=http://otelcol:4317 + - OTEL_EXPORTER_OTLP_PROTOCOL=grpc + - OTEL_EXPORTER_OTLP_INSECURE=true + - OTEL_RESOURCE_ATTRIBUTES=service.name=qryn + depends_on: + - clickhouse + qryn-collector: + image: ghcr.io/metrico/qryn-otel-collector:latest + volumes: + - ../qryncol.yml:/etc/otel/config.yaml + restart: on-failure + depends_on: + - clickhouse # Exports traces from clickhouse internal table to otel. chotel: @@ -78,7 +105,7 @@ services: restart: unless-stopped volumes: - ../prometheus:/etc/prometheus - - prom_data:/prometheus + - prom:/prometheus # https://opentelemetry.io/docs/collector/installation/#docker-compose otelcol: @@ -112,7 +139,6 @@ services: # to produce some metrics node-exporter: image: prom/node-exporter - container_name: node-exporter restart: unless-stopped volumes: - /proc:/host/proc:ro @@ -126,6 +152,7 @@ services: - '--web.max-requests=40' - '--collector.filesystem.mount-points-exclude=^/(sys|proc|dev|host|etc)($$|/)' + # for compatibility testing jaeger: image: jaegertracing/all-in-one:latest @@ -135,7 +162,9 @@ services: - GF_AUTH_ANONYMOUS_ENABLED=true - GF_AUTH_ANONYMOUS_ORG_ROLE=Admin - GF_AUTH_DISABLE_LOGIN_FORM=true - - GF_FEATURE_TOGGLES_ENABLE=traceqlEditor + - GF_FEATURE_TOGGLES_ENABLE=traceqlEditor,traceToMetrics,publicDashboards,tempoApmTable + - GF_ALERTING_ENABLED=false + - GF_USERS_ALLOW_SIGN_UP=false - GF_LOG_LEVEL=debug - GF_INSTALL_PLUGINS=grafana-clickhouse-datasource ports: diff --git a/dev/local/grafana/datasources.yaml b/dev/local/grafana/datasources.yaml index 004d6035..cc8c295d 100644 --- a/dev/local/grafana/datasources.yaml +++ b/dev/local/grafana/datasources.yaml @@ -50,10 +50,57 @@ datasources: uid: jaeger url: http://jaeger:16686 access: proxy + + # For qryn + - name: Loki qryn + type: loki + access: proxy + uid: loki-qryn + url: http://qryn:3100 + editable: true + jsonData: + derivedFields: + - datasourceUid: tempo-qryn + matcherRegex: "^.*?traceI[d|D]=(\\w+).*$" + name: traceId + url: '$${__value.raw}' + - datasourceUid: tempo-qryn + matcherRegex: "^.*?\"traceID\":\"(\\w+)\".*$" + name: traceID + url: '$${__value.raw}' + + - name: Tempo qryn + type: tempo + access: proxy + uid: tempo-qryn + url: http://qryn:3100 + editable: true jsonData: nodeGraph: enabled: true - traceQuery: - timeShiftEnabled: true - spanStartTimeShift: '1h' - spanEndTimeShift: '-1h' \ No newline at end of file + tracesToLogs: + datasourceUid: loki-qryn + filterByTraceID: false + spanEndTimeShift: "2000ms" + spanStartTimeShift: "-2000ms" + tags: ['job'] + tracesToMetrics: + datasourceUid: prom-qryn + tags: ['job'] + serviceMap: + datasourceUid: prom-qryn + lokiSearch: + datasourceUid: loki-qryn + + - name: PromQL qryn + type: prometheus + access: proxy + uid: prom-qryn + url: http://qryn:3100 + editable: true + basicAuth: false + jsonData: + httpMethod: GET + graphiteVersion: "1.1" + tlsAuth: false + tlsAuthWithCACert: false diff --git a/dev/local/otelcol.yml b/dev/local/otelcol.yml index 555ea3b9..a4eb2542 100644 --- a/dev/local/otelcol.yml +++ b/dev/local/otelcol.yml @@ -1,5 +1,3 @@ - - receivers: otlp: protocols: @@ -41,6 +39,16 @@ receivers: scrape_interval: 5s static_configs: - targets: [oteldb:9090] + + # For PromQL compliance tester + # https://github.com/prometheus/compliance/tree/main/promql#promql-compliance-tester + - job_name: 'demo' + static_configs: + - targets: + - 'demo.promlabs.com:10000' + - 'demo.promlabs.com:10001' + - 'demo.promlabs.com:10002' + processors: batch: @@ -53,6 +61,10 @@ exporters: endpoint: jaeger:4317 tls: insecure: true + otlp/qryn: + endpoint: qryn-collector:4317 + tls: + insecure: true prometheusremotewrite: endpoint: "http://prometheus:9090/api/v1/write" resource_to_telemetry_conversion: @@ -74,15 +86,15 @@ service: traces: receivers: [otlp] processors: [batch] - exporters: [otlp, otlp/jaeger] + exporters: [otlp, otlp/jaeger, otlp/qryn] metrics: receivers: [otlp, prometheus, hostmetrics] processors: [batch] - exporters: [otlp, prometheusremotewrite] + exporters: [otlp, prometheusremotewrite, otlp/qryn] logs: receivers: [otlp] processors: [batch] - exporters: [otlp, debug] + exporters: [otlp, debug, otlp/qryn] # https://github.com/open-telemetry/opentelemetry-collector/blob/main/docs/observability.md#how-we-expose-telemetry telemetry: metrics: diff --git a/dev/local/qryncol.yml b/dev/local/qryncol.yml new file mode 100644 index 00000000..1b416299 --- /dev/null +++ b/dev/local/qryncol.yml @@ -0,0 +1,89 @@ +receivers: + loki: + use_incoming_timestamp: true + protocols: + http: + endpoint: 0.0.0.0:3100 + grpc: + endpoint: 0.0.0.0:3200 + syslog: + protocol: rfc5424 + tcp: + listen_address: "0.0.0.0:5514" + fluentforward: + endpoint: 0.0.0.0:24224 + splunk_hec: + endpoint: 0.0.0.0:8088 + otlp: + protocols: + grpc: + endpoint: 0.0.0.0:4317 + http: + endpoint: 0.0.0.0:4318 + jaeger: + protocols: + grpc: + endpoint: 0.0.0.0:14250 + thrift_http: + endpoint: 0.0.0.0:14268 + zipkin: + endpoint: 0.0.0.0:9411 + skywalking: + protocols: + grpc: + endpoint: 0.0.0.0:11800 + http: + endpoint: 0.0.0.0:12800 + influxdb: + endpoint: 0.0.0.0:8086 +processors: + batch: + send_batch_size: 10000 + timeout: 5s + memory_limiter: + check_interval: 2s + limit_mib: 1800 + spike_limit_mib: 500 + +exporters: + qryn: + dsn: tcp://clickhouse:9000/cloki + timeout: 10s + sending_queue: + queue_size: 100 + retry_on_failure: + enabled: true + initial_interval: 5s + max_interval: 30s + max_elapsed_time: 300s + logs: + format: raw + otlp: + endpoint: localhost:4317 + tls: + insecure: true +extensions: + health_check: + pprof: + zpages: + memory_ballast: + size_mib: 1000 + +service: + telemetry: + logs: + level: "info" + extensions: [pprof, zpages, health_check] + pipelines: + logs: + receivers: [fluentforward, otlp, loki] + processors: [memory_limiter, batch] + exporters: [qryn] + traces: + receivers: [otlp, jaeger, zipkin, skywalking] + processors: [memory_limiter, batch] + exporters: [qryn] + metrics: + receivers: [otlp, influxdb] + processors: [memory_limiter, batch] + exporters: [qryn] From 69a62790f7330381836ee6e8deeecdaf56b3a41e Mon Sep 17 00:00:00 2001 From: tdakkota Date: Fri, 24 Nov 2023 20:19:01 +0300 Subject: [PATCH 037/112] feat(chstorage): insert all received metrics data --- internal/chstorage/columns_metrics.go | 226 +++++++++++++++++-- internal/chstorage/inserter.go | 2 +- internal/chstorage/inserter_metrics.go | 299 +++++++++++++++++++------ internal/chstorage/querier.go | 2 +- internal/chstorage/querier_metrics.go | 2 +- internal/chstorage/schema.go | 52 +++-- internal/chstorage/schema_metrics.go | 62 ++++- 7 files changed, 531 insertions(+), 114 deletions(-) diff --git a/internal/chstorage/columns_metrics.go b/internal/chstorage/columns_metrics.go index f34f4a78..eca23d63 100644 --- a/internal/chstorage/columns_metrics.go +++ b/internal/chstorage/columns_metrics.go @@ -1,48 +1,238 @@ package chstorage -import "github.com/ClickHouse/ch-go/proto" +import ( + "github.com/ClickHouse/ch-go/proto" +) + +type pointColumns struct { + name *proto.ColLowCardinality[string] + timestamp *proto.ColDateTime64 + + value proto.ColFloat64 -type metricColumns struct { - name *proto.ColLowCardinality[string] - timestamp *proto.ColDateTime64 - value proto.ColFloat64 attributes proto.ColStr resource proto.ColStr } -func newMetricColumns() *metricColumns { - return &metricColumns{ +func newPointColumns() *pointColumns { + return &pointColumns{ name: new(proto.ColStr).LowCardinality(), timestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), } } -func (c *metricColumns) StaticColumns() []string { - return []string{ - "name", - "timestamp", - "value", - "attributes", - "resource", +func (c *pointColumns) Input() proto.Input { + input := proto.Input{ + {Name: "name", Data: c.name}, + {Name: "timestamp", Data: c.timestamp}, + + {Name: "value", Data: c.value}, + + {Name: "attributes", Data: c.attributes}, + {Name: "resource", Data: c.resource}, } + return input } -func (c *metricColumns) Input() proto.Input { +func (c *pointColumns) Result() proto.Results { + return proto.Results{ + {Name: "name", Data: c.name}, + {Name: "timestamp", Data: c.timestamp}, + + {Name: "value", Data: &c.value}, + + {Name: "attributes", Data: &c.attributes}, + {Name: "resource", Data: &c.resource}, + } +} + +type histogramColumns struct { + name *proto.ColLowCardinality[string] + timestamp *proto.ColDateTime64 + + count proto.ColUInt64 + sum *proto.ColNullable[float64] + min *proto.ColNullable[float64] + max *proto.ColNullable[float64] + bucketCounts *proto.ColArr[uint64] + explicitBounds *proto.ColArr[float64] + + attributes proto.ColStr + resource proto.ColStr +} + +func newHistogramColumns() *histogramColumns { + return &histogramColumns{ + name: new(proto.ColStr).LowCardinality(), + timestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + + sum: new(proto.ColFloat64).Nullable(), + min: new(proto.ColFloat64).Nullable(), + max: new(proto.ColFloat64).Nullable(), + bucketCounts: new(proto.ColUInt64).Array(), + explicitBounds: new(proto.ColFloat64).Array(), + } +} + +func (c *histogramColumns) Input() proto.Input { input := proto.Input{ {Name: "name", Data: c.name}, {Name: "timestamp", Data: c.timestamp}, - {Name: "value", Data: c.value}, + + {Name: "histogram_count", Data: c.count}, + {Name: "histogram_sum", Data: c.sum}, + {Name: "histogram_min", Data: c.min}, + {Name: "histogram_max", Data: c.max}, + {Name: "histogram_bucket_counts", Data: c.bucketCounts}, + {Name: "histogram_explicit_bounds", Data: c.explicitBounds}, + {Name: "attributes", Data: c.attributes}, {Name: "resource", Data: c.resource}, } return input } -func (c *metricColumns) Result() proto.Results { +func (c *histogramColumns) Result() proto.Results { return proto.Results{ {Name: "name", Data: c.name}, {Name: "timestamp", Data: c.timestamp}, - {Name: "value", Data: &c.value}, + + {Name: "histogram_count", Data: &c.count}, + {Name: "histogram_sum", Data: c.sum}, + {Name: "histogram_min", Data: c.min}, + {Name: "histogram_max", Data: c.max}, + {Name: "histogram_bucket_counts", Data: c.bucketCounts}, + {Name: "histogram_explicit_bounds", Data: c.explicitBounds}, + + {Name: "attributes", Data: &c.attributes}, + {Name: "resource", Data: &c.resource}, + } +} + +type expHistogramColumns struct { + name *proto.ColLowCardinality[string] + timestamp *proto.ColDateTime64 + + count proto.ColUInt64 + sum *proto.ColNullable[float64] + min *proto.ColNullable[float64] + max *proto.ColNullable[float64] + scale proto.ColInt32 + zerocount proto.ColUInt64 + positiveOffset proto.ColInt32 + positiveBucketCounts *proto.ColArr[uint64] + negativeOffset proto.ColInt32 + negativeBucketCounts *proto.ColArr[uint64] + + attributes proto.ColStr + resource proto.ColStr +} + +func newExpHistogramColumns() *expHistogramColumns { + return &expHistogramColumns{ + name: new(proto.ColStr).LowCardinality(), + timestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + + sum: new(proto.ColFloat64).Nullable(), + min: new(proto.ColFloat64).Nullable(), + max: new(proto.ColFloat64).Nullable(), + positiveBucketCounts: new(proto.ColUInt64).Array(), + negativeBucketCounts: new(proto.ColUInt64).Array(), + } +} + +func (c *expHistogramColumns) Input() proto.Input { + input := proto.Input{ + {Name: "name", Data: c.name}, + {Name: "timestamp", Data: c.timestamp}, + + {Name: "exp_histogram_count", Data: c.count}, + {Name: "exp_histogram_sum", Data: c.sum}, + {Name: "exp_histogram_min", Data: c.min}, + {Name: "exp_histogram_max", Data: c.max}, + {Name: "exp_histogram_scale", Data: c.scale}, + {Name: "exp_histogram_zerocount", Data: c.zerocount}, + {Name: "exp_histogram_positive_offset", Data: c.positiveOffset}, + {Name: "exp_histogram_positive_bucket_counts", Data: c.positiveBucketCounts}, + {Name: "exp_histogram_negative_offset", Data: c.negativeOffset}, + {Name: "exp_histogram_negative_bucket_counts", Data: c.negativeBucketCounts}, + + {Name: "attributes", Data: c.attributes}, + {Name: "resource", Data: c.resource}, + } + return input +} + +func (c *expHistogramColumns) Result() proto.Results { + return proto.Results{ + {Name: "name", Data: c.name}, + {Name: "timestamp", Data: c.timestamp}, + + {Name: "exp_histogram_count", Data: &c.count}, + {Name: "exp_histogram_sum", Data: c.sum}, + {Name: "exp_histogram_min", Data: c.min}, + {Name: "exp_histogram_max", Data: c.max}, + {Name: "exp_histogram_scale", Data: &c.scale}, + {Name: "exp_histogram_zerocount", Data: &c.zerocount}, + {Name: "exp_histogram_positive_offset", Data: &c.positiveOffset}, + {Name: "exp_histogram_positive_bucket_counts", Data: c.positiveBucketCounts}, + {Name: "exp_histogram_negative_offset", Data: &c.negativeOffset}, + {Name: "exp_histogram_negative_bucket_counts", Data: c.negativeBucketCounts}, + + {Name: "attributes", Data: &c.attributes}, + {Name: "resource", Data: &c.resource}, + } +} + +type summaryColumns struct { + name *proto.ColLowCardinality[string] + timestamp *proto.ColDateTime64 + + count proto.ColUInt64 + sum proto.ColFloat64 + quantiles *proto.ColArr[float64] + values *proto.ColArr[float64] + + attributes proto.ColStr + resource proto.ColStr +} + +func newSummaryColumns() *summaryColumns { + return &summaryColumns{ + name: new(proto.ColStr).LowCardinality(), + timestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + + quantiles: new(proto.ColFloat64).Array(), + values: new(proto.ColFloat64).Array(), + } +} + +func (c *summaryColumns) Input() proto.Input { + input := proto.Input{ + {Name: "name", Data: c.name}, + {Name: "timestamp", Data: c.timestamp}, + + {Name: "summary_count", Data: c.count}, + {Name: "summary_sum", Data: c.sum}, + {Name: "summary_quantiles", Data: c.quantiles}, + {Name: "summary_values", Data: c.values}, + + {Name: "attributes", Data: c.attributes}, + {Name: "resource", Data: c.resource}, + } + return input +} + +func (c *summaryColumns) Result() proto.Results { + return proto.Results{ + {Name: "name", Data: c.name}, + {Name: "timestamp", Data: c.timestamp}, + + {Name: "summary_count", Data: &c.count}, + {Name: "summary_sum", Data: &c.sum}, + {Name: "summary_quantiles", Data: c.quantiles}, + {Name: "summary_values", Data: c.values}, + {Name: "attributes", Data: &c.attributes}, {Name: "resource", Data: &c.resource}, } diff --git a/internal/chstorage/inserter.go b/internal/chstorage/inserter.go index 3f379a58..9b9bfaaa 100644 --- a/internal/chstorage/inserter.go +++ b/internal/chstorage/inserter.go @@ -36,7 +36,7 @@ type InserterOptions struct { func (opts *InserterOptions) setDefaults() { if opts.Tables == (Tables{}) { - opts.Tables = defaultTables + opts.Tables = DefaultTables() } if opts.MeterProvider == nil { opts.MeterProvider = otel.GetMeterProvider() diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index cb7ca86f..090d1d6a 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -4,6 +4,8 @@ import ( "context" "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/chpool" + "github.com/ClickHouse/ch-go/proto" "github.com/go-faster/errors" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" @@ -12,101 +14,239 @@ import ( // ConsumeMetrics inserts given metrics. func (i *Inserter) ConsumeMetrics(ctx context.Context, metrics pmetric.Metrics) error { - var ( - points = newMetricColumns() - - labels = newLabelsColumns() - collectLabels = func(m pcommon.Map) { - m.Range(func(k string, v pcommon.Value) bool { - labels.name.Append(k) - // FIXME(tdakkota): annoying allocations - labels.value.Append(v.AsString()) - return true - }) - } - ) - - if err := i.mapMetrics(points, metrics, collectLabels); err != nil { + b := newMetricBatch() + if err := i.mapMetrics(b, metrics); err != nil { return errors.Wrap(err, "map metrics") } + if err := b.Insert(ctx, i.tables, i.ch); err != nil { + return errors.Wrap(err, "send batch") + } + return nil +} - { - grp, grpCtx := errgroup.WithContext(ctx) +type metricsBatch struct { + points *pointColumns + histograms *histogramColumns + expHistograms *expHistogramColumns + summaries *summaryColumns + labels *labelsColumns +} - grp.Go(func() error { - ctx := grpCtx +func newMetricBatch() *metricsBatch { + return &metricsBatch{ + points: newPointColumns(), + histograms: newHistogramColumns(), + expHistograms: newExpHistogramColumns(), + summaries: newSummaryColumns(), + labels: newLabelsColumns(), + } +} - input := points.Input() - if err := i.ch.Do(ctx, ch.Query{ - Body: input.Into(i.tables.Points), - Input: input, - }); err != nil { - return errors.Wrap(err, "insert points") - } - return nil - }) +func (b *metricsBatch) Insert(ctx context.Context, tables Tables, client *chpool.Pool) error { + grp, grpCtx := errgroup.WithContext(ctx) + + type columns interface { + Input() proto.Input + } + for _, table := range []struct { + name string + columns columns + }{ + {tables.Points, b.points}, + {tables.Histograms, b.histograms}, + {tables.ExpHistograms, b.expHistograms}, + {tables.Summaries, b.summaries}, + {tables.Labels, b.labels}, + } { + table := table grp.Go(func() error { ctx := grpCtx - input := labels.Input() - if err := i.ch.Do(ctx, ch.Query{ - Body: input.Into(i.tables.Labels), + input := table.columns.Input() + if err := client.Do(ctx, ch.Query{ + Body: input.Into(table.name), Input: input, }); err != nil { - return errors.Wrap(err, "insert labels") + return errors.Wrapf(err, "insert %q", table.name) } return nil }) - if err := grp.Wait(); err != nil { - return err + } + if err := grp.Wait(); err != nil { + return err + } + return nil +} + +func (b *metricsBatch) addPoints(name string, res pcommon.Map, slice pmetric.NumberDataPointSlice) error { + c := b.points + + for i := 0; i < slice.Len(); i++ { + point := slice.At(i) + ts := point.Timestamp().AsTime() + attrs := point.Attributes() + + var val float64 + switch typ := point.ValueType(); typ { + case pmetric.NumberDataPointValueTypeEmpty: + // Just ignore it. + continue + case pmetric.NumberDataPointValueTypeInt: + // TODO(tdakkota): check for overflow + val = float64(point.IntValue()) + case pmetric.NumberDataPointValueTypeDouble: + val = point.DoubleValue() + default: + return errors.Errorf("unexpected metric %q value type: %v", name, typ) } + + b.addLabels(attrs) + c.name.Append(name) + c.timestamp.Append(ts) + c.value.Append(val) + c.attributes.Append(encodeAttributes(attrs)) + c.resource.Append(encodeAttributes(res)) } + return nil +} +func (b *metricsBatch) addHistogramPoints(name string, res pcommon.Map, slice pmetric.HistogramDataPointSlice) error { + c := b.histograms + for i := 0; i < slice.Len(); i++ { + point := slice.At(i) + ts := point.Timestamp().AsTime() + attrs := point.Attributes() + count := point.Count() + sum := proto.Nullable[float64]{ + Set: point.HasSum(), + Value: point.Sum(), + } + min := proto.Nullable[float64]{ + Set: point.HasMin(), + Value: point.Min(), + } + max := proto.Nullable[float64]{ + Set: point.HasMax(), + Value: point.Max(), + } + bucketCounts := point.BucketCounts().AsRaw() + explicitBounds := point.ExplicitBounds().AsRaw() + + b.addLabels(attrs) + c.name.Append(name) + c.timestamp.Append(ts) + c.count.Append(count) + c.sum.Append(sum) + c.min.Append(min) + c.max.Append(max) + c.bucketCounts.Append(bucketCounts) + c.explicitBounds.Append(explicitBounds) + c.attributes.Append(encodeAttributes(attrs)) + c.resource.Append(encodeAttributes(res)) + } return nil } -func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics, collectLabels func(attrs pcommon.Map)) error { +func (b *metricsBatch) addExpHistogramPoints(name string, res pcommon.Map, slice pmetric.ExponentialHistogramDataPointSlice) error { var ( - addPoints = func( - name string, - res pcommon.Map, - slices pmetric.NumberDataPointSlice, - ) error { - for i := 0; i < slices.Len(); i++ { - point := slices.At(i) - ts := point.Timestamp().AsTime() - attrs := point.Attributes() - - var val float64 - switch typ := point.ValueType(); typ { - case pmetric.NumberDataPointValueTypeEmpty: - // Just ignore it. - continue - case pmetric.NumberDataPointValueTypeInt: - // TODO(tdakkota): check for overflow - val = float64(point.IntValue()) - case pmetric.NumberDataPointValueTypeDouble: - val = point.DoubleValue() - default: - return errors.Errorf("unexpected metric %q value type: %v", name, typ) - } + c = b.expHistograms + mapBuckets = func(b pmetric.ExponentialHistogramDataPointBuckets) (offset int32, counts []uint64) { + offset = b.Offset() + counts = b.BucketCounts().AsRaw() + return offset, counts + } + ) + for i := 0; i < slice.Len(); i++ { + point := slice.At(i) + ts := point.Timestamp().AsTime() + attrs := point.Attributes() + count := point.Count() + sum := proto.Nullable[float64]{ + Set: point.HasSum(), + Value: point.Sum(), + } + min := proto.Nullable[float64]{ + Set: point.HasMin(), + Value: point.Min(), + } + max := proto.Nullable[float64]{ + Set: point.HasMax(), + Value: point.Max(), + } + scale := point.Scale() + zerocount := point.ZeroCount() - collectLabels(attrs) - c.name.Append(name) - c.timestamp.Append(ts) - c.value.Append(val) - c.attributes.Append(encodeAttributes(attrs)) - c.resource.Append(encodeAttributes(res)) - } - return nil + positiveOffset, positiveBucketCounts := mapBuckets(point.Positive()) + negativeOffset, negativeBucketCounts := mapBuckets(point.Negative()) + + b.addLabels(attrs) + c.name.Append(name) + c.timestamp.Append(ts) + c.count.Append(count) + c.sum.Append(sum) + c.min.Append(min) + c.max.Append(max) + c.scale.Append(scale) + c.zerocount.Append(zerocount) + c.positiveOffset.Append(positiveOffset) + c.positiveBucketCounts.Append(positiveBucketCounts) + c.negativeOffset.Append(negativeOffset) + c.negativeBucketCounts.Append(negativeBucketCounts) + c.attributes.Append(encodeAttributes(attrs)) + c.resource.Append(encodeAttributes(res)) + } + return nil +} + +func (b *metricsBatch) addSummaryPoints(name string, res pcommon.Map, slice pmetric.SummaryDataPointSlice) error { + c := b.summaries + for i := 0; i < slice.Len(); i++ { + point := slice.At(i) + ts := point.Timestamp().AsTime() + attrs := point.Attributes() + count := point.Count() + sum := point.Sum() + var ( + qv = point.QuantileValues() + + quantiles = make([]float64, qv.Len()) + values = make([]float64, qv.Len()) + ) + for i := 0; i < qv.Len(); i++ { + p := qv.At(i) + + quantiles[i] = p.Quantile() + values[i] = p.Value() } - resMetrics = metrics.ResourceMetrics() - ) + b.addLabels(attrs) + c.name.Append(name) + c.timestamp.Append(ts) + c.count.Append(count) + c.sum.Append(sum) + c.quantiles.Append(quantiles) + c.values.Append(values) + c.attributes.Append(encodeAttributes(attrs)) + c.resource.Append(encodeAttributes(res)) + } + return nil +} + +func (b *metricsBatch) addLabels(m pcommon.Map) { + m.Range(func(k string, v pcommon.Value) bool { + b.labels.name.Append(k) + // FIXME(tdakkota): annoying allocations + b.labels.value.Append(v.AsString()) + return true + }) +} + +func (i *Inserter) mapMetrics(b *metricsBatch, metrics pmetric.Metrics) error { + resMetrics := metrics.ResourceMetrics() for i := 0; i < resMetrics.Len(); i++ { resMetric := resMetrics.At(i) resAttrs := resMetric.Resource().Attributes() - collectLabels(resAttrs) + b.addLabels(resAttrs) scopeMetrics := resMetric.ScopeMetrics() for i := 0; i < scopeMetrics.Len(); i++ { @@ -120,16 +260,29 @@ func (i *Inserter) mapMetrics(c *metricColumns, metrics pmetric.Metrics, collect switch typ := record.Type(); typ { case pmetric.MetricTypeGauge: gauge := record.Gauge() - if err := addPoints(name, resAttrs, gauge.DataPoints()); err != nil { + if err := b.addPoints(name, resAttrs, gauge.DataPoints()); err != nil { return err } case pmetric.MetricTypeSum: sum := record.Sum() - if err := addPoints(name, resAttrs, sum.DataPoints()); err != nil { + if err := b.addPoints(name, resAttrs, sum.DataPoints()); err != nil { + return err + } + case pmetric.MetricTypeHistogram: + hist := record.Histogram() + if err := b.addHistogramPoints(name, resAttrs, hist.DataPoints()); err != nil { + return err + } + case pmetric.MetricTypeExponentialHistogram: + hist := record.ExponentialHistogram() + if err := b.addExpHistogramPoints(name, resAttrs, hist.DataPoints()); err != nil { + return err + } + case pmetric.MetricTypeSummary: + summary := record.Summary() + if err := b.addSummaryPoints(name, resAttrs, summary.DataPoints()); err != nil { return err } - case pmetric.MetricTypeHistogram, pmetric.MetricTypeExponentialHistogram, pmetric.MetricTypeSummary: - // FIXME(tdakkota): ignore for now. default: return errors.Errorf("unexpected metric %q type %v", name, typ) } diff --git a/internal/chstorage/querier.go b/internal/chstorage/querier.go index b44d19a0..a9233a21 100644 --- a/internal/chstorage/querier.go +++ b/internal/chstorage/querier.go @@ -30,7 +30,7 @@ type QuerierOptions struct { func (opts *QuerierOptions) setDefaults() { if opts.Tables == (Tables{}) { - opts.Tables = defaultTables + opts.Tables = DefaultTables() } if opts.MeterProvider == nil { opts.MeterProvider = otel.GetMeterProvider() diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index d69958b5..819ee3fc 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -275,7 +275,7 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series var ( set = map[seriesKey]seriesWithLabels{} - c = newMetricColumns() + c = newPointColumns() ) if err := p.ch.Do(ctx, ch.Query{ Body: query, diff --git a/internal/chstorage/schema.go b/internal/chstorage/schema.go index cb0f7b1d..9d7b4ac8 100644 --- a/internal/chstorage/schema.go +++ b/internal/chstorage/schema.go @@ -13,42 +13,57 @@ type Tables struct { Spans string Tags string - Points string - Labels string + Points string + Histograms string + ExpHistograms string + Summaries string + Labels string } // Validate checks table names -func (t Tables) Validate() error { - validateTableName := func(name string) error { - if name == "" { +func (t *Tables) Validate() error { + return t.Each(func(name *string) error { + if *name == "" { return errors.New("table name must be non-empty") } return nil - } + }) +} +// Each calls given callback for each table. +func (t *Tables) Each(cb func(name *string) error) error { for _, table := range []struct { - name string + field *string fieldName string }{ - {t.Spans, "Spans"}, - {t.Tags, "Tags"}, + {&t.Spans, "Spans"}, + {&t.Tags, "Tags"}, - {t.Points, "Points"}, - {t.Labels, "Labels"}, + {&t.Points, "Points"}, + {&t.Histograms, "Histograms"}, + {&t.ExpHistograms, "ExpHistograms"}, + {&t.Summaries, "Summaries"}, + {&t.Labels, "Labels"}, } { - if err := validateTableName(table.name); err != nil { + if err := cb(table.field); err != nil { return errors.Wrapf(err, "table %s", table.fieldName) } } return nil } -var defaultTables = Tables{ - Spans: "traces_spans", - Tags: "traces_tags", +// DefaultTables returns default tables. +func DefaultTables() Tables { + return Tables{ + Spans: "traces_spans", + Tags: "traces_tags", - Points: "metrics_points", - Labels: "metrics_labels", + Points: "metrics_points", + Histograms: "metrics_histograms", + ExpHistograms: "metrics_exp_histograms", + Summaries: "metrics_summaries", + Labels: "metrics_labels", + } } type chClient interface { @@ -70,6 +85,9 @@ func (t Tables) Create(ctx context.Context, c chClient) error { {t.Tags, tagsSchema}, {t.Points, pointsSchema}, + {t.Histograms, histogramsSchema}, + {t.ExpHistograms, expHistogramsSchema}, + {t.Summaries, summariesSchema}, {t.Labels, labelsSchema}, } { if err := c.Do(ctx, ch.Query{ diff --git a/internal/chstorage/schema_metrics.go b/internal/chstorage/schema_metrics.go index a4c6b0cf..9742ba16 100644 --- a/internal/chstorage/schema_metrics.go +++ b/internal/chstorage/schema_metrics.go @@ -10,14 +10,70 @@ import ( const ( pointsSchema = `CREATE TABLE IF NOT EXISTS %s ( - name LowCardinality(String), - timestamp DateTime64(9), - value Float64, + name LowCardinality(String), + timestamp DateTime64(9), + + value Float64, + + attributes String, + resource String + ) + ENGINE = MergeTree() + ORDER BY timestamp;` + histogramsSchema = `CREATE TABLE IF NOT EXISTS %s + ( + name LowCardinality(String), + timestamp DateTime64(9), + + histogram_count UInt64, + histogram_sum Nullable(Float64), + histogram_min Nullable(Float64), + histogram_max Nullable(Float64), + histogram_bucket_counts Array(UInt64), + histogram_explicit_bounds Array(Float64), + attributes String, resource String ) ENGINE = MergeTree() ORDER BY timestamp;` + expHistogramsSchema = `CREATE TABLE IF NOT EXISTS %s + ( + name LowCardinality(String), + timestamp DateTime64(9), + + exp_histogram_count UInt64, + exp_histogram_sum Nullable(Float64), + exp_histogram_min Nullable(Float64), + exp_histogram_max Nullable(Float64), + exp_histogram_scale Int32, + exp_histogram_zerocount UInt64, + exp_histogram_positive_offset Int32, + exp_histogram_positive_bucket_counts Array(UInt64), + exp_histogram_negative_offset Int32, + exp_histogram_negative_bucket_counts Array(UInt64), + + attributes String, + resource String + ) + ENGINE = MergeTree() + ORDER BY timestamp;` + summariesSchema = `CREATE TABLE IF NOT EXISTS %s + ( + name LowCardinality(String), + timestamp DateTime64(9), + + summary_count UInt64, + summary_sum Float64, + summary_quantiles Array(Float64), + summary_values Array(Float64), + + attributes String, + resource String + ) + ENGINE = MergeTree() + ORDER BY timestamp;` + labelsSchema = `CREATE TABLE IF NOT EXISTS %s ( name LowCardinality(String), From e992694b6a336f9dc22b412ffa124f2bd1df61e2 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Fri, 24 Nov 2023 20:19:32 +0300 Subject: [PATCH 038/112] chore(integration): update tests due to `chstorage` changes --- integration/prome2e/ch_test.go | 15 +++++++-------- integration/tempoe2e/ch_test.go | 15 +++++++-------- 2 files changed, 14 insertions(+), 16 deletions(-) diff --git a/integration/prome2e/ch_test.go b/integration/prome2e/ch_test.go index 341087d5..37de5ca7 100644 --- a/integration/prome2e/ch_test.go +++ b/integration/prome2e/ch_test.go @@ -62,14 +62,13 @@ func TestCH(t *testing.T) { t.Fatal(err) } - prefix := "traces_" + strings.ReplaceAll(uuid.NewString(), "-", "") - tables := chstorage.Tables{ - Spans: prefix + "_spans", - Tags: prefix + "_tags", - - Points: prefix + "_points", - Labels: prefix + "_labels", - } + prefix := strings.ReplaceAll(uuid.NewString(), "-", "") + tables := chstorage.DefaultTables() + tables.Each(func(name *string) error { + old := *name + *name = prefix + "_" + old + return nil + }) t.Logf("Test tables prefix: %s", prefix) require.NoError(t, tables.Create(ctx, c)) diff --git a/integration/tempoe2e/ch_test.go b/integration/tempoe2e/ch_test.go index 9e718ef8..28b99c8e 100644 --- a/integration/tempoe2e/ch_test.go +++ b/integration/tempoe2e/ch_test.go @@ -62,14 +62,13 @@ func TestCH(t *testing.T) { t.Fatal(err) } - prefix := "traces_" + strings.ReplaceAll(uuid.NewString(), "-", "") - tables := chstorage.Tables{ - Spans: prefix + "_spans", - Tags: prefix + "_tags", - - Points: prefix + "_points", - Labels: prefix + "_labels", - } + prefix := strings.ReplaceAll(uuid.NewString(), "-", "") + tables := chstorage.DefaultTables() + tables.Each(func(name *string) error { + old := *name + *name = prefix + "_" + old + return nil + }) t.Logf("Test tables prefix: %s", prefix) require.NoError(t, tables.Create(ctx, c)) From 323ed1cbd24c87216757142fec8f3c599ef2a678 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Fri, 24 Nov 2023 20:21:01 +0300 Subject: [PATCH 039/112] fix(prome2e): collect labels from all metrics --- integration/prome2e/prom_e2e.go | 23 ++++++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/integration/prome2e/prom_e2e.go b/integration/prome2e/prom_e2e.go index 1dbc1496..74190a30 100644 --- a/integration/prome2e/prom_e2e.go +++ b/integration/prome2e/prom_e2e.go @@ -88,13 +88,34 @@ func (s *BatchSet) addMetric(metric pmetric.Metric) error { } return nil case pmetric.MetricTypeHistogram: + points := metric.Histogram().DataPoints() + for i := 0; i < points.Len(); i++ { + point := points.At(i) + s.addLabels(point.Attributes()) + s.addTimestamp(point.Timestamp()) + } + return nil case pmetric.MetricTypeExponentialHistogram: + points := metric.ExponentialHistogram().DataPoints() + for i := 0; i < points.Len(); i++ { + point := points.At(i) + s.addLabels(point.Attributes()) + s.addTimestamp(point.Timestamp()) + } + return nil case pmetric.MetricTypeSummary: + points := metric.Summary().DataPoints() + for i := 0; i < points.Len(); i++ { + point := points.At(i) + s.addLabels(point.Attributes()) + s.addTimestamp(point.Timestamp()) + } + return nil case pmetric.MetricTypeEmpty: + return nil default: return errors.Errorf("unexpected type %v", t) } - return nil } func (s *BatchSet) addTimestamp(ts pcommon.Timestamp) { From 09788838ee2f052462a8c8c198a5dd5ada8c6a44 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 00:32:59 +0300 Subject: [PATCH 040/112] feat(chstorage): draft log inserter --- cmd/oteldb/storage.go | 2 +- internal/chstorage/columns_logs.go | 92 +++++++++++++++++++++++++++++ internal/chstorage/inserter.go | 20 ++++--- internal/chstorage/inserter_logs.go | 54 +++++++++++++++++ internal/chstorage/schema.go | 8 +++ internal/chstorage/schema_logs.go | 22 +++++++ 6 files changed, 190 insertions(+), 8 deletions(-) create mode 100644 internal/chstorage/columns_logs.go create mode 100644 internal/chstorage/inserter_logs.go create mode 100644 internal/chstorage/schema_logs.go diff --git a/cmd/oteldb/storage.go b/cmd/oteldb/storage.go index 66dff4f4..04d53696 100644 --- a/cmd/oteldb/storage.go +++ b/cmd/oteldb/storage.go @@ -279,7 +279,7 @@ func setupCH( return otelStorage{ logQuerier: nil, - logInserter: nil, + logInserter: inserter, traceQuerier: querier, traceInserter: inserter, metricsQuerier: querier, diff --git a/internal/chstorage/columns_logs.go b/internal/chstorage/columns_logs.go new file mode 100644 index 00000000..5cbb4036 --- /dev/null +++ b/internal/chstorage/columns_logs.go @@ -0,0 +1,92 @@ +package chstorage + +import ( + "github.com/ClickHouse/ch-go/proto" + + "github.com/go-faster/oteldb/internal/logstorage" + "github.com/go-faster/oteldb/internal/otelstorage" +) + +type logColumns struct { + timestamp *proto.ColDateTime64 + observedTimestamp *proto.ColDateTime64 + flags proto.ColUInt32 + severityNumber proto.ColInt32 + severityText *proto.ColLowCardinality[string] + body proto.ColStr + traceID proto.ColRawOf[otelstorage.TraceID] + spanID proto.ColRawOf[otelstorage.SpanID] + attributes proto.ColStr + resource proto.ColStr + scopeName proto.ColStr + scopeVersion proto.ColStr + scopeAttributes proto.ColStr +} + +func newLogColumns() *logColumns { + return &logColumns{ + timestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + observedTimestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), + severityText: new(proto.ColStr).LowCardinality(), + } +} + +func (c *logColumns) StaticColumns() []string { + var cols []string + for _, col := range c.Input() { + cols = append(cols, col.Name) + } + return cols +} + +func (c *logColumns) AddRow(r logstorage.Record) { + c.timestamp.Append(r.Timestamp.AsTime()) + c.observedTimestamp.Append(r.ObservedTimestamp.AsTime()) + c.flags.Append(uint32(r.Flags)) + c.severityNumber.Append(int32(r.SeverityNumber)) + c.severityText.Append(r.SeverityText) + c.body.Append(r.Body) + c.traceID.Append(r.TraceID) + c.spanID.Append(r.SpanID) + c.attributes.Append(encodeAttributes(r.Attrs.AsMap())) + c.resource.Append(encodeAttributes(r.ResourceAttrs.AsMap())) + c.scopeName.Append(r.ScopeName) + c.scopeVersion.Append(r.ScopeVersion) + c.scopeAttributes.Append(encodeAttributes(r.ScopeAttrs.AsMap())) +} + +func (c *logColumns) Input() proto.Input { + return proto.Input{ + {Name: "timestamp", Data: c.timestamp}, + {Name: "observed_timestamp", Data: c.observedTimestamp}, + {Name: "flags", Data: c.flags}, + {Name: "severity_number", Data: c.severityNumber}, + {Name: "severity_text", Data: c.severityText}, + {Name: "body", Data: c.body}, + {Name: "trace_id", Data: c.traceID}, + {Name: "span_id", Data: c.spanID}, + {Name: "attributes", Data: c.attributes}, + {Name: "resource", Data: c.resource}, + {Name: "scope_name", Data: c.scopeName}, + {Name: "scope_version", Data: c.scopeVersion}, + {Name: "scope_attributes", Data: c.scopeAttributes}, + } +} + +func (c *logColumns) Result() proto.Results { + return proto.Results{ + {Name: "timestamp", Data: c.timestamp}, + {Name: "observed_timestamp", Data: c.observedTimestamp}, + {Name: "flags", Data: &c.flags}, + {Name: "severity_number", Data: &c.severityNumber}, + {Name: "severity_text", Data: c.severityText}, + {Name: "body", Data: &c.body}, + {Name: "trace_id", Data: &c.traceID}, + {Name: "span_id", Data: &c.spanID}, + {Name: "attributes", Data: &c.attributes}, + {Name: "resource", Data: &c.resource}, + {Name: "scope_name", Data: &c.scopeName}, + {Name: "scope_version", Data: &c.scopeVersion}, + {Name: "scope_attributes", Data: &c.scopeAttributes}, + } +} diff --git a/internal/chstorage/inserter.go b/internal/chstorage/inserter.go index 9b9bfaaa..0ea89d91 100644 --- a/internal/chstorage/inserter.go +++ b/internal/chstorage/inserter.go @@ -18,8 +18,9 @@ type Inserter struct { ch *chpool.Pool tables Tables - insertedSpans metric.Int64Counter - insertedTags metric.Int64Counter + insertedSpans metric.Int64Counter + insertedTags metric.Int64Counter + insertedRecords metric.Int64Counter tracer trace.Tracer } @@ -59,12 +60,17 @@ func NewInserter(c *chpool.Pool, opts InserterOptions) (*Inserter, error) { if err != nil { return nil, errors.Wrap(err, "create inserted_tags") } + insertedRecords, err := meter.Int64Counter("chstorage.traces.inserted_records") + if err != nil { + return nil, errors.Wrap(err, "create inserted_records") + } return &Inserter{ - ch: c, - tables: opts.Tables, - insertedSpans: insertedSpans, - insertedTags: insertedTags, - tracer: opts.TracerProvider.Tracer("chstorage.Inserter"), + ch: c, + tables: opts.Tables, + insertedSpans: insertedSpans, + insertedTags: insertedTags, + insertedRecords: insertedRecords, + tracer: opts.TracerProvider.Tracer("chstorage.Inserter"), }, nil } diff --git a/internal/chstorage/inserter_logs.go b/internal/chstorage/inserter_logs.go new file mode 100644 index 00000000..6bf049ed --- /dev/null +++ b/internal/chstorage/inserter_logs.go @@ -0,0 +1,54 @@ +package chstorage + +import ( + "context" + + "github.com/ClickHouse/ch-go" + "github.com/go-faster/errors" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/trace" + + "github.com/go-faster/oteldb/internal/logstorage" +) + +func (i *Inserter) mapRecords(c *logColumns, records []logstorage.Record) { + for _, r := range records { + c.AddRow(r) + } +} + +// InsertLogLabels inserts given set of labels to the storage. +func (i *Inserter) InsertLogLabels(context.Context, map[logstorage.Label]struct{}) error { + // No-op. + // TODO(ernado): do we really need this or can just use materialized view? + return nil +} + +// InsertRecords inserts given records. +func (i *Inserter) InsertRecords(ctx context.Context, records []logstorage.Record) (rerr error) { + table := i.tables.Logs + ctx, span := i.tracer.Start(ctx, "InsertRecords", trace.WithAttributes( + attribute.Int("chstorage.records_count", len(records)), + attribute.String("chstorage.table", table), + )) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } else { + i.insertedRecords.Add(ctx, int64(len(records))) + } + span.End() + }() + + logs := newLogColumns() + i.mapRecords(logs, records) + + if err := i.ch.Do(ctx, ch.Query{ + Body: logs.Input().Into(table), + Input: logs.Input(), + }); err != nil { + return errors.Wrap(err, "insert records") + } + + return nil +} diff --git a/internal/chstorage/schema.go b/internal/chstorage/schema.go index 9d7b4ac8..23760c8a 100644 --- a/internal/chstorage/schema.go +++ b/internal/chstorage/schema.go @@ -18,6 +18,8 @@ type Tables struct { ExpHistograms string Summaries string Labels string + + Logs string } // Validate checks table names @@ -44,6 +46,8 @@ func (t *Tables) Each(cb func(name *string) error) error { {&t.ExpHistograms, "ExpHistograms"}, {&t.Summaries, "Summaries"}, {&t.Labels, "Labels"}, + + {&t.Logs, "Logs"}, } { if err := cb(table.field); err != nil { return errors.Wrapf(err, "table %s", table.fieldName) @@ -63,6 +67,8 @@ func DefaultTables() Tables { ExpHistograms: "metrics_exp_histograms", Summaries: "metrics_summaries", Labels: "metrics_labels", + + Logs: "logs", } } @@ -89,6 +95,8 @@ func (t Tables) Create(ctx context.Context, c chClient) error { {t.ExpHistograms, expHistogramsSchema}, {t.Summaries, summariesSchema}, {t.Labels, labelsSchema}, + + {t.Logs, logsSchema}, } { if err := c.Do(ctx, ch.Query{ Body: fmt.Sprintf(s.query, s.name), diff --git a/internal/chstorage/schema_logs.go b/internal/chstorage/schema_logs.go new file mode 100644 index 00000000..ffb3db46 --- /dev/null +++ b/internal/chstorage/schema_logs.go @@ -0,0 +1,22 @@ +package chstorage + +const ( + logsSchema = `CREATE TABLE IF NOT EXISTS %s +( + timestamp DateTime64(9), + timestamp_observed DateTime64(9), + flags UInt32, + severity_text String, + severity_number Int32, + body String, + trace_id UUID, + span_id UInt64, + attributes String, + resource String, + scope_name String, + scope_version String, + scope_attributes String, +) +ENGINE = MergeTree() +ORDER BY (timestamp);` +) From b9c1517c6951e96cc98f7b2f3b0b05bed7c90df7 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 00:40:06 +0300 Subject: [PATCH 041/112] fix: setup all table names --- cmd/oteldb/storage.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/cmd/oteldb/storage.go b/cmd/oteldb/storage.go index 04d53696..d5b308ac 100644 --- a/cmd/oteldb/storage.go +++ b/cmd/oteldb/storage.go @@ -252,8 +252,11 @@ func setupCH( } tables := chstorage.Tables{ - Spans: "traces_spans", - Tags: "traces_tags", + Spans: "traces_spans", + Tags: "traces_tags", + Logs: "logs", + Points: "metrics_points", + Labels: "metrics_labels", } if err := tables.Create(ctx, c); err != nil { return store, errors.Wrap(err, "create tables") From 7b8981cc0fc34ec33b2f801a71296b8d35f83c5d Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 00:43:22 +0300 Subject: [PATCH 042/112] chore(lint): wrap err --- internal/chstorage/inserter_metrics.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index 090d1d6a..491f5f7e 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -73,8 +73,9 @@ func (b *metricsBatch) Insert(ctx context.Context, tables Tables, client *chpool }) } if err := grp.Wait(); err != nil { - return err + return errors.Wrap(err, "insert") } + return nil } From 9630391aa129afe4c0bfe1d9611d1aa4460eb436 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 00:46:11 +0300 Subject: [PATCH 043/112] fix: use DefaultTables --- cmd/oteldb/storage.go | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/cmd/oteldb/storage.go b/cmd/oteldb/storage.go index d5b308ac..03380b83 100644 --- a/cmd/oteldb/storage.go +++ b/cmd/oteldb/storage.go @@ -251,13 +251,7 @@ func setupCH( return store, errors.Wrap(err, "migrate") } - tables := chstorage.Tables{ - Spans: "traces_spans", - Tags: "traces_tags", - Logs: "logs", - Points: "metrics_points", - Labels: "metrics_labels", - } + tables := chstorage.DefaultTables() if err := tables.Create(ctx, c); err != nil { return store, errors.Wrap(err, "create tables") } From 5103260a9fd8d4e60aed1cc76c2b345cdb1ad626 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 01:41:13 +0300 Subject: [PATCH 044/112] feat(dev): add otelfaker --- cmd/otelfaker/main.go | 7 ++++++- dev/local/ch/docker-compose.yml | 11 +++++++++++ otelfaker.Dockerfile | 17 +++++++++++++++++ 3 files changed, 34 insertions(+), 1 deletion(-) create mode 100644 otelfaker.Dockerfile diff --git a/cmd/otelfaker/main.go b/cmd/otelfaker/main.go index 0c182bd7..60731de7 100644 --- a/cmd/otelfaker/main.go +++ b/cmd/otelfaker/main.go @@ -2,6 +2,7 @@ package main import ( "context" + "os" "time" "github.com/go-faster/errors" @@ -63,7 +64,11 @@ func main() { otelgrpc.WithTracerProvider(m.TracerProvider()), otelgrpc.WithMeterProvider(m.MeterProvider()), } - conn, err := grpc.DialContext(ctx, "oteldb.faster.svc.cluster.local:4317", + target := os.Getenv("OTEL_TARGET") + if target == "" { + target = "oteldb.faster.svc.cluster.local:4317" + } + conn, err := grpc.DialContext(ctx, target, grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithStatsHandler(otelgrpc.NewClientHandler(otelOptions...)), ) diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index c577d1bf..0596d7c9 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -84,6 +84,17 @@ services: depends_on: - clickhouse + faker: + image: go-faster/oteldb/faker + build: + context: ../../../ + dockerfile: otelfaker.Dockerfile + environment: + - OTEL_TARGET=otelcol:4317 + depends_on: + - oteldb + - otelcol + # Generates traces. k6-tracing: image: ghcr.io/grafana/xk6-client-tracing:v0.0.2 diff --git a/otelfaker.Dockerfile b/otelfaker.Dockerfile new file mode 100644 index 00000000..7707687e --- /dev/null +++ b/otelfaker.Dockerfile @@ -0,0 +1,17 @@ +FROM golang:latest as builder + +WORKDIR /app + +COPY go.mod go.sum ./ +RUN go mod download + +COPY . ./ +RUN CGO_ENABLED=0 GOOS=linux go build -o /app/otelfaker ./cmd/otelfaker + +FROM alpine:latest +RUN apk --no-cache add ca-certificates + +WORKDIR /app +COPY --from=builder /app/otelfaker /otelfaker + +ENTRYPOINT ["/otelfaker"] From f15f2edafd1c8c988254128bfe640bfcc52f3552 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 01:54:27 +0300 Subject: [PATCH 045/112] fix(chstorage): fix log schema --- internal/chstorage/schema_logs.go | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/internal/chstorage/schema_logs.go b/internal/chstorage/schema_logs.go index ffb3db46..ece8d9a8 100644 --- a/internal/chstorage/schema_logs.go +++ b/internal/chstorage/schema_logs.go @@ -3,19 +3,19 @@ package chstorage const ( logsSchema = `CREATE TABLE IF NOT EXISTS %s ( - timestamp DateTime64(9), - timestamp_observed DateTime64(9), - flags UInt32, - severity_text String, - severity_number Int32, - body String, - trace_id UUID, - span_id UInt64, - attributes String, - resource String, - scope_name String, - scope_version String, - scope_attributes String, + timestamp DateTime64(9), + observed_timestamp DateTime64(9), + flags UInt32, + severity_text String, + severity_number Int32, + body String, + trace_id FixedString(16), + span_id FixedString(8), + attributes String, + resource String, + scope_name String, + scope_version String, + scope_attributes String, ) ENGINE = MergeTree() ORDER BY (timestamp);` From 02a15fe7845271ced13018ca6bae0f2fa47561d6 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 01:54:33 +0300 Subject: [PATCH 046/112] chore(dev): rename faker service --- dev/local/ch/docker-compose.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index 0596d7c9..d5c23410 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -84,7 +84,7 @@ services: depends_on: - clickhouse - faker: + otelfaker: image: go-faster/oteldb/faker build: context: ../../../ From 403d19c76f9f3c4c0dfae99cf72f40899242de67 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 12:45:33 +0300 Subject: [PATCH 047/112] feat: improve clickhouse client logging --- cmd/oteldb/storage.go | 15 ++++++++++++++- go.mod | 6 +++--- 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/cmd/oteldb/storage.go b/cmd/oteldb/storage.go index 03380b83..e9772463 100644 --- a/cmd/oteldb/storage.go +++ b/cmd/oteldb/storage.go @@ -16,6 +16,7 @@ import ( "github.com/prometheus/prometheus/storage" "go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ytzap "go.ytsaurus.tech/library/go/core/log/zap" "go.ytsaurus.tech/yt/go/migrate" "go.ytsaurus.tech/yt/go/ypath" @@ -220,8 +221,20 @@ func setupCH( } pass, _ := u.User.Password() + chLogger := lg.Named("ch") + { + var lvl zapcore.Level + if v := os.Getenv("CH_LOG_LEVEL"); v != "" { + if err := lvl.UnmarshalText([]byte(v)); err != nil { + return store, errors.Wrap(err, "parse log level") + } + } else { + lvl = lg.Level() + } + chLogger = chLogger.WithOptions(zap.IncreaseLevel(lvl)) + } opts := ch.Options{ - Logger: lg.Named("ch"), + Logger: chLogger, Address: u.Host, Database: strings.TrimPrefix(u.Path, "/"), User: u.User.Username(), diff --git a/go.mod b/go.mod index 9974b9f5..057d960f 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module github.com/go-faster/oteldb go 1.21.0 require ( - github.com/ClickHouse/ch-go v0.58.2 + github.com/ClickHouse/ch-go v0.60.0 github.com/Masterminds/sprig/v3 v3.2.3 github.com/cenkalti/backoff/v4 v4.2.1 github.com/cespare/xxhash/v2 v2.2.0 @@ -80,7 +80,7 @@ require ( github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/dennwc/varint v1.0.0 // indirect github.com/dlclark/regexp2 v1.10.0 // indirect - github.com/dmarkham/enumer v1.5.8 // indirect + github.com/dmarkham/enumer v1.5.9 // indirect github.com/docker/distribution v2.8.2+incompatible // indirect github.com/docker/go-connections v0.4.0 // indirect github.com/docker/go-units v0.5.0 // indirect @@ -112,7 +112,7 @@ require ( github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/jpillora/backoff v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.17.2 // indirect + github.com/klauspost/compress v1.17.3 // indirect github.com/klauspost/cpuid/v2 v2.0.9 // indirect github.com/knadh/koanf v1.5.0 // indirect github.com/knadh/koanf/v2 v2.0.1 // indirect From 4ce2ff1abb5df50d90eefa64fdd411b2aa27449f Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 14:22:15 +0300 Subject: [PATCH 048/112] feat(otelschema): add semconv schema parser --- go.mod | 1 + internal/otelschema/_testdata/model/README.md | 41 + .../otelschema/_testdata/model/exception.yaml | 33 + .../_testdata/model/faas-common.yaml | 77 + .../otelschema/_testdata/model/general.yaml | 85 + .../_testdata/model/http-common.yaml | 87 + .../_testdata/model/logs/events.yaml | 18 + .../_testdata/model/logs/general.yaml | 19 + .../_testdata/model/logs/log-exception.yaml | 16 + .../model/logs/log-feature_flag.yaml | 11 + .../_testdata/model/logs/media.yaml | 49 + .../_testdata/model/logs/mobile-events.yaml | 72 + .../model/metrics/database-metrics.yaml | 107 + .../_testdata/model/metrics/faas-metrics.yaml | 81 + .../_testdata/model/metrics/http.yaml | 119 + .../metrics/jvm-metrics-experimental.yaml | 70 + .../_testdata/model/metrics/jvm-metrics.yaml | 168 ++ .../_testdata/model/metrics/rpc-metrics.yaml | 122 + .../model/metrics/system-metrics.yaml | 501 ++++ .../otelschema/_testdata/model/network.yaml | 28 + .../_testdata/model/registry/client.yaml | 28 + .../_testdata/model/registry/cloud.yaml | 179 ++ .../_testdata/model/registry/code.yaml | 41 + .../_testdata/model/registry/container.yaml | 86 + .../_testdata/model/registry/db.yaml | 424 +++ .../model/registry/deprecated/http.yaml | 71 + .../model/registry/deprecated/network.yaml | 100 + .../_testdata/model/registry/destination.yaml | 23 + .../_testdata/model/registry/device.yaml | 46 + .../_testdata/model/registry/disk.yaml | 17 + .../_testdata/model/registry/error.yaml | 35 + .../_testdata/model/registry/host.yaml | 122 + .../_testdata/model/registry/http.yaml | 135 + .../_testdata/model/registry/k8s.yaml | 140 + .../_testdata/model/registry/messaging.yaml | 245 ++ .../_testdata/model/registry/network.yaml | 194 ++ .../_testdata/model/registry/oci.yaml | 21 + .../_testdata/model/registry/os.yaml | 69 + .../_testdata/model/registry/process.yaml | 78 + .../_testdata/model/registry/rpc.yaml | 190 ++ .../_testdata/model/registry/server.yaml | 28 + .../_testdata/model/registry/source.yaml | 23 + .../_testdata/model/registry/thread.yaml | 17 + .../_testdata/model/registry/tls.yaml | 165 ++ .../_testdata/model/registry/url.yaml | 41 + .../_testdata/model/registry/user-agent.yaml | 13 + .../_testdata/model/resource/android.yaml | 14 + .../_testdata/model/resource/browser.yaml | 56 + .../_testdata/model/resource/cloud.yaml | 13 + .../resource/cloud_provider/aws/ecs.yaml | 42 + .../resource/cloud_provider/aws/eks.yaml | 12 + .../resource/cloud_provider/aws/logs.yaml | 39 + .../cloud_provider/gcp/cloud_run.yaml | 23 + .../resource/cloud_provider/gcp/gce.yaml | 22 + .../model/resource/cloud_provider/heroku.yaml | 25 + .../_testdata/model/resource/container.yaml | 22 + .../resource/deployment_environment.yaml | 21 + .../_testdata/model/resource/device.yaml | 11 + .../_testdata/model/resource/faas.yaml | 63 + .../_testdata/model/resource/host.yaml | 37 + .../_testdata/model/resource/k8s.yaml | 98 + .../_testdata/model/resource/os.yaml | 16 + .../_testdata/model/resource/process.yaml | 42 + .../_testdata/model/resource/service.yaml | 23 + .../model/resource/service_experimental.yaml | 37 + .../_testdata/model/resource/telemetry.yaml | 57 + .../resource/telemetry_experimental.yaml | 20 + .../_testdata/model/resource/webengine.yaml | 23 + .../model/scope/exporter/exporter.yaml | 30 + .../otelschema/_testdata/model/session.yaml | 26 + .../_testdata/model/trace/aws/lambda.yaml | 14 + .../_testdata/model/trace/cloudevents.yaml | 36 + .../_testdata/model/trace/compatibility.yaml | 21 + .../_testdata/model/trace/database.yaml | 259 ++ .../model/trace/exporter/exporter.yaml | 21 + .../_testdata/model/trace/faas.yaml | 144 + .../_testdata/model/trace/feature-flag.yaml | 34 + .../_testdata/model/trace/http.yaml | 91 + .../model/trace/instrumentation/aws-sdk.yml | 516 ++++ .../model/trace/instrumentation/graphql.yml | 32 + .../_testdata/model/trace/messaging.yaml | 182 ++ .../otelschema/_testdata/model/trace/rpc.yaml | 129 + .../model/trace/trace-exception.yaml | 38 + internal/otelschema/_testdata/model/url.yaml | 13 + .../_testdata/model/version.properties | 1 + internal/otelschema/generate.go | 3 + internal/otelschema/group.go | 2389 +++++++++++++++++ internal/otelschema/group_test.go | 34 + internal/otelschema/otelschema.go | 10 + internal/otelschema/schema.yml | 112 + 90 files changed, 9087 insertions(+) create mode 100644 internal/otelschema/_testdata/model/README.md create mode 100644 internal/otelschema/_testdata/model/exception.yaml create mode 100644 internal/otelschema/_testdata/model/faas-common.yaml create mode 100644 internal/otelschema/_testdata/model/general.yaml create mode 100644 internal/otelschema/_testdata/model/http-common.yaml create mode 100644 internal/otelschema/_testdata/model/logs/events.yaml create mode 100644 internal/otelschema/_testdata/model/logs/general.yaml create mode 100644 internal/otelschema/_testdata/model/logs/log-exception.yaml create mode 100644 internal/otelschema/_testdata/model/logs/log-feature_flag.yaml create mode 100644 internal/otelschema/_testdata/model/logs/media.yaml create mode 100644 internal/otelschema/_testdata/model/logs/mobile-events.yaml create mode 100644 internal/otelschema/_testdata/model/metrics/database-metrics.yaml create mode 100644 internal/otelschema/_testdata/model/metrics/faas-metrics.yaml create mode 100644 internal/otelschema/_testdata/model/metrics/http.yaml create mode 100644 internal/otelschema/_testdata/model/metrics/jvm-metrics-experimental.yaml create mode 100644 internal/otelschema/_testdata/model/metrics/jvm-metrics.yaml create mode 100644 internal/otelschema/_testdata/model/metrics/rpc-metrics.yaml create mode 100644 internal/otelschema/_testdata/model/metrics/system-metrics.yaml create mode 100644 internal/otelschema/_testdata/model/network.yaml create mode 100644 internal/otelschema/_testdata/model/registry/client.yaml create mode 100644 internal/otelschema/_testdata/model/registry/cloud.yaml create mode 100644 internal/otelschema/_testdata/model/registry/code.yaml create mode 100644 internal/otelschema/_testdata/model/registry/container.yaml create mode 100644 internal/otelschema/_testdata/model/registry/db.yaml create mode 100644 internal/otelschema/_testdata/model/registry/deprecated/http.yaml create mode 100644 internal/otelschema/_testdata/model/registry/deprecated/network.yaml create mode 100644 internal/otelschema/_testdata/model/registry/destination.yaml create mode 100644 internal/otelschema/_testdata/model/registry/device.yaml create mode 100644 internal/otelschema/_testdata/model/registry/disk.yaml create mode 100644 internal/otelschema/_testdata/model/registry/error.yaml create mode 100644 internal/otelschema/_testdata/model/registry/host.yaml create mode 100644 internal/otelschema/_testdata/model/registry/http.yaml create mode 100644 internal/otelschema/_testdata/model/registry/k8s.yaml create mode 100644 internal/otelschema/_testdata/model/registry/messaging.yaml create mode 100644 internal/otelschema/_testdata/model/registry/network.yaml create mode 100644 internal/otelschema/_testdata/model/registry/oci.yaml create mode 100644 internal/otelschema/_testdata/model/registry/os.yaml create mode 100644 internal/otelschema/_testdata/model/registry/process.yaml create mode 100644 internal/otelschema/_testdata/model/registry/rpc.yaml create mode 100644 internal/otelschema/_testdata/model/registry/server.yaml create mode 100644 internal/otelschema/_testdata/model/registry/source.yaml create mode 100644 internal/otelschema/_testdata/model/registry/thread.yaml create mode 100644 internal/otelschema/_testdata/model/registry/tls.yaml create mode 100644 internal/otelschema/_testdata/model/registry/url.yaml create mode 100644 internal/otelschema/_testdata/model/registry/user-agent.yaml create mode 100644 internal/otelschema/_testdata/model/resource/android.yaml create mode 100644 internal/otelschema/_testdata/model/resource/browser.yaml create mode 100644 internal/otelschema/_testdata/model/resource/cloud.yaml create mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/aws/ecs.yaml create mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/aws/eks.yaml create mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/aws/logs.yaml create mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/gcp/cloud_run.yaml create mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/gcp/gce.yaml create mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/heroku.yaml create mode 100644 internal/otelschema/_testdata/model/resource/container.yaml create mode 100644 internal/otelschema/_testdata/model/resource/deployment_environment.yaml create mode 100644 internal/otelschema/_testdata/model/resource/device.yaml create mode 100644 internal/otelschema/_testdata/model/resource/faas.yaml create mode 100644 internal/otelschema/_testdata/model/resource/host.yaml create mode 100644 internal/otelschema/_testdata/model/resource/k8s.yaml create mode 100644 internal/otelschema/_testdata/model/resource/os.yaml create mode 100644 internal/otelschema/_testdata/model/resource/process.yaml create mode 100644 internal/otelschema/_testdata/model/resource/service.yaml create mode 100644 internal/otelschema/_testdata/model/resource/service_experimental.yaml create mode 100644 internal/otelschema/_testdata/model/resource/telemetry.yaml create mode 100644 internal/otelschema/_testdata/model/resource/telemetry_experimental.yaml create mode 100644 internal/otelschema/_testdata/model/resource/webengine.yaml create mode 100644 internal/otelschema/_testdata/model/scope/exporter/exporter.yaml create mode 100644 internal/otelschema/_testdata/model/session.yaml create mode 100644 internal/otelschema/_testdata/model/trace/aws/lambda.yaml create mode 100644 internal/otelschema/_testdata/model/trace/cloudevents.yaml create mode 100644 internal/otelschema/_testdata/model/trace/compatibility.yaml create mode 100644 internal/otelschema/_testdata/model/trace/database.yaml create mode 100644 internal/otelschema/_testdata/model/trace/exporter/exporter.yaml create mode 100644 internal/otelschema/_testdata/model/trace/faas.yaml create mode 100644 internal/otelschema/_testdata/model/trace/feature-flag.yaml create mode 100644 internal/otelschema/_testdata/model/trace/http.yaml create mode 100644 internal/otelschema/_testdata/model/trace/instrumentation/aws-sdk.yml create mode 100644 internal/otelschema/_testdata/model/trace/instrumentation/graphql.yml create mode 100644 internal/otelschema/_testdata/model/trace/messaging.yaml create mode 100644 internal/otelschema/_testdata/model/trace/rpc.yaml create mode 100644 internal/otelschema/_testdata/model/trace/trace-exception.yaml create mode 100644 internal/otelschema/_testdata/model/url.yaml create mode 100644 internal/otelschema/_testdata/model/version.properties create mode 100644 internal/otelschema/generate.go create mode 100644 internal/otelschema/group.go create mode 100644 internal/otelschema/group_test.go create mode 100644 internal/otelschema/otelschema.go create mode 100644 internal/otelschema/schema.yml diff --git a/go.mod b/go.mod index 057d960f..2703a269 100644 --- a/go.mod +++ b/go.mod @@ -56,6 +56,7 @@ require ( golang.org/x/sync v0.5.0 golang.org/x/tools v0.15.0 google.golang.org/grpc v1.59.0 + sigs.k8s.io/yaml v1.3.0 ) require ( diff --git a/internal/otelschema/_testdata/model/README.md b/internal/otelschema/_testdata/model/README.md new file mode 100644 index 00000000..2fc6f39b --- /dev/null +++ b/internal/otelschema/_testdata/model/README.md @@ -0,0 +1,41 @@ +# YAML Model for Semantic Conventions + +The YAML descriptions of semantic convention contained in this directory are intended to +be used by the various OpenTelemetry language implementations to aid in automatic +generation of semantics-related code. + +⚠ If you want to read the semantic conventions and not edit them, please see +the generated markdown output in the [docs](../docs/README.md) folder. + +## Writing semantic conventions + +Semantic conventions for the spec MUST adhere to the +[attribute naming](../docs/general/attribute-naming.md), +[attribute requirement level](../docs/general/attribute-requirement-level.md), +and [metric requirement level](https://github.com/open-telemetry/opentelemetry-specification/tree/v1.26.0/specification/metrics/metric-requirement-level.md) conventions. + +Refer to the [syntax](https://github.com/open-telemetry/build-tools/tree/v0.23.0/semantic-conventions/syntax.md) +for how to write the YAML files for semantic conventions and what the YAML properties mean. + +A schema file for VS code is configured in the `/.vscode/settings.json` of this +repository, enabling auto-completion and additional checks. Refer to +[the generator README](https://github.com/open-telemetry/build-tools/tree/v0.23.0/semantic-conventions/README.md) for what extension you need. + +## Generating markdown + +These YAML files are used by the make target `table-generation` to generate consistently +formatted Markdown tables for all semantic conventions in the specification. Run it from the root of this repository using the command + +``` +make table-generation +``` + +For more information, see the [semantic convention generator](https://github.com/open-telemetry/build-tools/tree/v0.23.0/semantic-conventions) +in the OpenTelemetry build tools repository. +Using this build tool, it is also possible to generate code for use in OpenTelemetry +language projects. + +See also: + +* [Markdown Tables](https://github.com/open-telemetry/build-tools/tree/main/semantic-conventions#markdown-tables) +* [Code Generator](https://github.com/open-telemetry/build-tools/tree/main/semantic-conventions#code-generator) diff --git a/internal/otelschema/_testdata/model/exception.yaml b/internal/otelschema/_testdata/model/exception.yaml new file mode 100644 index 00000000..9f47fb67 --- /dev/null +++ b/internal/otelschema/_testdata/model/exception.yaml @@ -0,0 +1,33 @@ +groups: + - id: exception + type: span + prefix: exception + brief: > + This document defines the shared attributes used to + report a single exception associated with a span or log. + attributes: + - id: type + type: string + brief: > + The type of the exception (its fully-qualified class name, if applicable). + The dynamic type of the exception should be preferred over the static type + in languages that support it. + examples: ["java.net.ConnectException", "OSError"] + - id: message + type: string + brief: The exception message. + examples: ["Division by zero", "Can't convert 'int' object to str implicitly"] + - id: stacktrace + type: string + brief: > + A stacktrace as a string in the natural representation for the language runtime. + The representation is to be determined and documented by each language SIG. + examples: 'Exception in thread "main" java.lang.RuntimeException: Test exception\n + at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n + at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n + at com.example.GenerateTrace.main(GenerateTrace.java:5)' + + constraints: + - any_of: + - "exception.type" + - "exception.message" diff --git a/internal/otelschema/_testdata/model/faas-common.yaml b/internal/otelschema/_testdata/model/faas-common.yaml new file mode 100644 index 00000000..7f903bb5 --- /dev/null +++ b/internal/otelschema/_testdata/model/faas-common.yaml @@ -0,0 +1,77 @@ +groups: + - id: attributes.faas.common + type: attribute_group + brief: "Describes FaaS attributes." + prefix: faas + attributes: + - id: trigger + brief: 'Type of the trigger which caused this function invocation.' + type: + allow_custom_values: false + members: + - id: datasource + value: 'datasource' + brief: 'A response to some data source operation such as a database or filesystem read/write' + - id: http + value: 'http' + brief: 'To provide an answer to an inbound HTTP request' + - id: pubsub + value: 'pubsub' + brief: 'A function is set to be executed when messages are sent to a messaging system' + - id: timer + value: 'timer' + brief: 'A function is scheduled to be executed regularly' + - id: other + value: 'other' + brief: 'If none of the others apply' + - id: invoked_name + type: string + requirement_level: required + brief: > + The name of the invoked function. + note: > + SHOULD be equal to the `faas.name` resource attribute of the + invoked function. + examples: 'my-function' + - id: invoked_provider + type: + allow_custom_values: true + members: + - id: 'alibaba_cloud' + value: 'alibaba_cloud' + brief: 'Alibaba Cloud' + - id: 'aws' + value: 'aws' + brief: 'Amazon Web Services' + - id: 'azure' + value: 'azure' + brief: 'Microsoft Azure' + - id: 'gcp' + value: 'gcp' + brief: 'Google Cloud Platform' + - id: 'tencent_cloud' + value: 'tencent_cloud' + brief: 'Tencent Cloud' + requirement_level: required + brief: > + The cloud provider of the invoked function. + note: > + SHOULD be equal to the `cloud.provider` resource attribute of the + invoked function. + - id: invoked_region + type: string + requirement_level: + conditionally_required: > + For some cloud providers, like AWS or GCP, the region in which a + function is hosted is essential to uniquely identify the function + and also part of its endpoint. Since it's part of the endpoint + being called, the region is always known to clients. In these cases, + `faas.invoked_region` MUST be set accordingly. If the region is + unknown to the client or not required for identifying the invoked + function, setting `faas.invoked_region` is optional. + brief: > + The cloud region of the invoked function. + note: > + SHOULD be equal to the `cloud.region` resource attribute of the + invoked function. + examples: 'eu-central-1' diff --git a/internal/otelschema/_testdata/model/general.yaml b/internal/otelschema/_testdata/model/general.yaml new file mode 100644 index 00000000..e3bad3d4 --- /dev/null +++ b/internal/otelschema/_testdata/model/general.yaml @@ -0,0 +1,85 @@ +groups: + - id: general.client + type: attribute_group + brief: > + General client attributes. + attributes: + - ref: client.address + - ref: client.port + - id: general.server + type: attribute_group + brief: > + General server attributes. + attributes: + - ref: server.address + - ref: server.port + - id: general.source + type: attribute_group + brief: > + General source attributes. + attributes: + - ref: source.address + - ref: source.port + - id: general.destination + type: attribute_group + brief: > + General destination attributes. + attributes: + - ref: destination.address + - ref: destination.port + - id: peer + prefix: peer + type: span + brief: "Operations that access some remote service." + attributes: + - id: service + type: string + brief: > + The [`service.name`](/docs/resource/README.md#service) + of the remote service. SHOULD be equal to the actual `service.name` + resource attribute of the remote service if any. + examples: "AuthTokenCache" + - id: identity + prefix: enduser + type: span + brief: > + These attributes may be used for any operation with an authenticated and/or authorized enduser. + attributes: + - id: id + type: string + brief: > + Username or client_id extracted from the access token or + [Authorization](https://tools.ietf.org/html/rfc7235#section-4.2) + header in the inbound request from outside the system. + examples: 'username' + - id: role + type: string + brief: 'Actual/assumed role the client is making the request under extracted from token or application security context.' + examples: 'admin' + - id: scope + type: string + brief: > + Scopes or granted authorities the client currently possesses extracted from token + or application security context. The value would come from the scope associated + with an [OAuth 2.0 Access Token](https://tools.ietf.org/html/rfc6749#section-3.3) + or an attribute value in a [SAML 2.0 Assertion](http://docs.oasis-open.org/security/saml/Post2.0/sstc-saml-tech-overview-2.0.html). + examples: 'read:message, write:files' + - id: thread + type: span + brief: > + These attributes may be used for any operation to store information about a thread that started a span. + attributes: + - ref: thread.id + - ref: thread.name + - id: code + type: span + brief: > + These attributes allow to report this unit of code and therefore to provide more context about the span. + attributes: + - ref: code.function + - ref: code.namespace + - ref: code.filepath + - ref: code.lineno + - ref: code.column + - ref: code.stacktrace + requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/http-common.yaml b/internal/otelschema/_testdata/model/http-common.yaml new file mode 100644 index 00000000..2187943d --- /dev/null +++ b/internal/otelschema/_testdata/model/http-common.yaml @@ -0,0 +1,87 @@ +groups: + - id: attributes.http.common + type: attribute_group + brief: "Describes HTTP attributes." + attributes: + - ref: http.request.method + requirement_level: required + - ref: http.response.status_code + requirement_level: + conditionally_required: If and only if one was received/sent. + - ref: error.type + requirement_level: + conditionally_required: If request has ended with an error. + examples: ['timeout', 'java.net.UnknownHostException', 'server_certificate_invalid', '500'] + note: | + If the request fails with an error before response status code was sent or received, + `error.type` SHOULD be set to exception type (its fully-qualified class name, if applicable) + or a component-specific low cardinality error identifier. + + If response status code was sent or received and status indicates an error according to [HTTP span status definition](/docs/http/http-spans.md), + `error.type` SHOULD be set to the status code number (represented as a string), an exception type (if thrown) or a component-specific error identifier. + + The `error.type` value SHOULD be predictable and SHOULD have low cardinality. + Instrumentations SHOULD document the list of errors they report. + + The cardinality of `error.type` within one instrumentation library SHOULD be low, but + telemetry consumers that aggregate data from multiple instrumentation libraries and applications + should be prepared for `error.type` to have high cardinality at query time, when no + additional filters are applied. + + If the request has completed successfully, instrumentations SHOULD NOT set `error.type`. + - ref: network.protocol.name + examples: ['http', 'spdy'] + requirement_level: + conditionally_required: If not `http` and `network.protocol.version` is set. + - ref: network.protocol.version + examples: ['1.0', '1.1', '2', '3'] + + - id: attributes.http.client + type: attribute_group + brief: 'HTTP Client attributes' + extends: attributes.http.common + attributes: + - ref: server.address + requirement_level: required + brief: > + Host identifier of the ["URI origin"](https://www.rfc-editor.org/rfc/rfc9110.html#name-uri-origin) HTTP request is sent to. + note: > + If an HTTP client request is explicitly made to an IP address, e.g. `http://x.x.x.x:8080`, then + `server.address` SHOULD be the IP address `x.x.x.x`. A DNS lookup SHOULD NOT be used. + - ref: server.port + requirement_level: required + brief: > + Port identifier of the ["URI origin"](https://www.rfc-editor.org/rfc/rfc9110.html#name-uri-origin) HTTP request is sent to. + - ref: url.scheme + requirement_level: opt_in + examples: ["http", "https"] + + - id: attributes.http.server + type: attribute_group + brief: 'HTTP Server attributes' + extends: attributes.http.common + attributes: + - ref: http.route + requirement_level: + conditionally_required: If and only if it's available + - ref: server.address + brief: > + Name of the local HTTP server that received the request. + note: > + See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). + - ref: server.port + brief: > + Port of the local HTTP server that received the request. + note: > + See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). + requirement_level: + conditionally_required: If `server.address` is set. + - ref: url.scheme + requirement_level: required + examples: ["http", "https"] + note: > + The scheme of the original client request, if known + (e.g. from [Forwarded#proto](https://developer.mozilla.org/docs/Web/HTTP/Headers/Forwarded#proto), + [X-Forwarded-Proto](https://developer.mozilla.org/docs/Web/HTTP/Headers/X-Forwarded-Proto), + or a similar header). + Otherwise, the scheme of the immediate peer request. diff --git a/internal/otelschema/_testdata/model/logs/events.yaml b/internal/otelschema/_testdata/model/logs/events.yaml new file mode 100644 index 00000000..3942d432 --- /dev/null +++ b/internal/otelschema/_testdata/model/logs/events.yaml @@ -0,0 +1,18 @@ +groups: + - id: event + type: attribute_group + prefix: event + brief: > + This document defines attributes for Events represented using Log Records. + attributes: + - id: name + type: string + requirement_level: required + brief: > + Identifies the class / type of event. + note: > + Event names are subject to the same rules as [attribute names](https://github.com/open-telemetry/opentelemetry-specification/tree/v1.26.0/specification/common/attribute-naming.md). + Notably, event names are namespaced to avoid collisions and provide a clean + separation of semantics for events in separate domains like browser, mobile, and + kubernetes. + examples: ['browser.mouse.click', 'device.app.lifecycle'] diff --git a/internal/otelschema/_testdata/model/logs/general.yaml b/internal/otelschema/_testdata/model/logs/general.yaml new file mode 100644 index 00000000..48da970c --- /dev/null +++ b/internal/otelschema/_testdata/model/logs/general.yaml @@ -0,0 +1,19 @@ +groups: + - id: log.record + prefix: log.record + type: attribute_group + brief: > + The attributes described in this section are rather generic. They may be used in any Log Record they apply to. + attributes: + - id: uid + type: string + requirement_level: opt_in + brief: > + A unique identifier for the Log Record. + note: > + If an id is provided, other log records with the same id will be considered duplicates and can be removed safely. + This means, that two distinguishable log records MUST have different values. + + The id MAY be an [Universally Unique Lexicographically Sortable Identifier (ULID)](https://github.com/ulid/spec), + but other identifiers (e.g. UUID) may be used as needed. + examples: ["01ARZ3NDEKTSV4RRFFQ69G5FAV"] diff --git a/internal/otelschema/_testdata/model/logs/log-exception.yaml b/internal/otelschema/_testdata/model/logs/log-exception.yaml new file mode 100644 index 00000000..d0b25155 --- /dev/null +++ b/internal/otelschema/_testdata/model/logs/log-exception.yaml @@ -0,0 +1,16 @@ +groups: + - id: log-exception + type: attribute_group + prefix: exception + brief: > + This document defines attributes for exceptions represented using Log + Records. + attributes: + - ref: exception.type + - ref: exception.message + - ref: exception.stacktrace + + constraints: + - any_of: + - "exception.type" + - "exception.message" diff --git a/internal/otelschema/_testdata/model/logs/log-feature_flag.yaml b/internal/otelschema/_testdata/model/logs/log-feature_flag.yaml new file mode 100644 index 00000000..6d902423 --- /dev/null +++ b/internal/otelschema/_testdata/model/logs/log-feature_flag.yaml @@ -0,0 +1,11 @@ +groups: + - id: log-feature_flag + type: event + prefix: feature_flag + brief: > + This document defines attributes for feature flag evaluations + represented using Log Records. + attributes: + - ref: feature_flag.key + - ref: feature_flag.provider_name + - ref: feature_flag.variant diff --git a/internal/otelschema/_testdata/model/logs/media.yaml b/internal/otelschema/_testdata/model/logs/media.yaml new file mode 100644 index 00000000..b8d1c299 --- /dev/null +++ b/internal/otelschema/_testdata/model/logs/media.yaml @@ -0,0 +1,49 @@ +groups: + - id: attributes.log + prefix: log + type: attribute_group + brief: "Describes Log attributes" + attributes: + - id: iostream + requirement_level: opt_in + brief: > + The stream associated with the log. See below for a list of well-known values. + type: + allow_custom_values: false + members: + - id: stdout + value: 'stdout' + brief: 'Logs from stdout stream' + - id: stderr + value: 'stderr' + brief: 'Events from stderr stream' + - id: attributes.log.file + prefix: log.file + type: attribute_group + brief: > + A file to which log was emitted. + attributes: + - id: name + type: string + requirement_level: recommended + brief: > + The basename of the file. + examples: ["audit.log"] + - id: path + type: string + requirement_level: opt_in + brief: > + The full path to the file. + examples: [ "/var/log/mysql/audit.log" ] + - id: name_resolved + type: string + requirement_level: opt_in + brief: > + The basename of the file, with symlinks resolved. + examples: [ "uuid.log" ] + - id: path_resolved + type: string + requirement_level: opt_in + brief: > + The full path to the file, with symlinks resolved. + examples: [ "/var/lib/docker/uuid.log" ] diff --git a/internal/otelschema/_testdata/model/logs/mobile-events.yaml b/internal/otelschema/_testdata/model/logs/mobile-events.yaml new file mode 100644 index 00000000..d6f49e70 --- /dev/null +++ b/internal/otelschema/_testdata/model/logs/mobile-events.yaml @@ -0,0 +1,72 @@ +groups: + - id: ios.lifecycle.events + type: event + prefix: ios + name: device.app.lifecycle + brief: > + This event represents an occurrence of a lifecycle transition on the iOS platform. + attributes: + - id: state + requirement_level: "required" + note: > + The iOS lifecycle states are defined in the [UIApplicationDelegate documentation](https://developer.apple.com/documentation/uikit/uiapplicationdelegate#1656902), + and from which the `OS terminology` column values are derived. + brief: > + This attribute represents the state the application has transitioned into at the occurrence of the event. + type: + allow_custom_values: false + members: + - id: active + value: 'active' + brief: > + The app has become `active`. Associated with UIKit notification `applicationDidBecomeActive`. + - id: inactive + value: 'inactive' + brief: > + The app is now `inactive`. Associated with UIKit notification `applicationWillResignActive`. + - id: background + value: 'background' + brief: > + The app is now in the background. + This value is associated with UIKit notification `applicationDidEnterBackground`. + - id: foreground + value: 'foreground' + brief: > + The app is now in the foreground. + This value is associated with UIKit notification `applicationWillEnterForeground`. + - id: terminate + value: 'terminate' + brief: > + The app is about to terminate. Associated with UIKit notification `applicationWillTerminate`. + - id: android.lifecycle.events + type: event + prefix: android + name: device.app.lifecycle + brief: > + This event represents an occurrence of a lifecycle transition on the Android platform. + attributes: + - id: state + requirement_level: required + brief: > + This attribute represents the state the application has transitioned into at the occurrence of the event. + note: > + The Android lifecycle states are defined in [Activity lifecycle callbacks](https://developer.android.com/guide/components/activities/activity-lifecycle#lc), + and from which the `OS identifiers` are derived. + type: + allow_custom_values: false + members: + - id: created + value: 'created' + brief: > + Any time before Activity.onResume() or, if the app has no Activity, Context.startService() + has been called in the app for the first time. + - id: background + value: 'background' + brief: > + Any time after Activity.onPause() or, if the app has no Activity, + Context.stopService() has been called when the app was in the foreground state. + - id: foreground + value: 'foreground' + brief: > + Any time after Activity.onResume() or, if the app has no Activity, + Context.startService() has been called when the app was in either the created or background states. diff --git a/internal/otelschema/_testdata/model/metrics/database-metrics.yaml b/internal/otelschema/_testdata/model/metrics/database-metrics.yaml new file mode 100644 index 00000000..3df6d86a --- /dev/null +++ b/internal/otelschema/_testdata/model/metrics/database-metrics.yaml @@ -0,0 +1,107 @@ +groups: + - id: attributes.db + type: attribute_group + brief: Describes Database attributes + attributes: + - id: state + type: + allow_custom_values: false + members: + - id: idle + value: 'idle' + - id: used + value: 'used' + requirement_level: required + brief: "The state of a connection in the pool" + examples: ["idle"] + - id: pool.name + type: string + requirement_level: required + brief: > + The name of the connection pool; unique within the instrumented application. + In case the connection pool implementation doesn't provide a name, + then the [db.connection_string](/docs/database/database-spans.md#connection-level-attributes) + should be used + examples: ["myDataSource"] + + - id: metric.db.client.connections.usage + type: metric + metric_name: db.client.connections.usage + brief: "The number of connections that are currently in state described by the `state` attribute" + instrument: updowncounter + unit: "{connection}" + attributes: + - ref: state + - ref: pool.name + + - id: metric.db.client.connections.idle.max + type: metric + metric_name: db.client.connections.idle.max + brief: "The maximum number of idle open connections allowed" + instrument: updowncounter + unit: "{connection}" + attributes: + - ref: pool.name + + - id: metric.db.client.connections.idle.min + type: metric + metric_name: db.client.connections.idle.min + brief: "The minimum number of idle open connections allowed" + instrument: updowncounter + unit: "{connection}" + attributes: + - ref: pool.name + + - id: metric.db.client.connections.max + type: metric + metric_name: db.client.connections.max + brief: "The maximum number of open connections allowed" + instrument: updowncounter + unit: "{connection}" + attributes: + - ref: pool.name + + - id: metric.db.client.connections.pending_requests + type: metric + metric_name: db.client.connections.pending_requests + brief: "The number of pending requests for an open connection, cumulative for the entire pool" + instrument: updowncounter + unit: "{request}" + attributes: + - ref: pool.name + + - id: metric.db.client.connections.timeouts + type: metric + metric_name: db.client.connections.timeouts + brief: "The number of connection timeouts that have occurred trying to obtain a connection from the pool" + instrument: counter + unit: "{timeout}" + attributes: + - ref: pool.name + + - id: metric.db.client.connections.create_time + type: metric + metric_name: db.client.connections.create_time + brief: "The time it took to create a new connection" + instrument: histogram + unit: "ms" + attributes: + - ref: pool.name + + - id: metric.db.client.connections.wait_time + type: metric + metric_name: db.client.connections.wait_time + brief: "The time it took to obtain an open connection from the pool" + instrument: histogram + unit: "ms" + attributes: + - ref: pool.name + + - id: metric.db.client.connections.use_time + type: metric + metric_name: db.client.connections.use_time + brief: "The time between borrowing a connection and returning it to the pool" + instrument: histogram + unit: "ms" + attributes: + - ref: pool.name diff --git a/internal/otelschema/_testdata/model/metrics/faas-metrics.yaml b/internal/otelschema/_testdata/model/metrics/faas-metrics.yaml new file mode 100644 index 00000000..d68119f3 --- /dev/null +++ b/internal/otelschema/_testdata/model/metrics/faas-metrics.yaml @@ -0,0 +1,81 @@ +groups: + - id: metric.faas.invoke_duration + type: metric + metric_name: faas.invoke_duration + brief: "Measures the duration of the function's logic execution" + instrument: histogram + unit: "s" + attributes: + - ref: faas.trigger + + - id: metric.faas.init_duration + type: metric + metric_name: faas.init_duration + brief: "Measures the duration of the function's initialization, such as a cold start" + instrument: histogram + unit: "s" + attributes: + - ref: faas.trigger + + - id: metric.faas.coldstarts + type: metric + metric_name: faas.coldstarts + brief: "Number of invocation cold starts" + instrument: counter + unit: "{coldstart}" + attributes: + - ref: faas.trigger + + - id: metric.faas.errors + type: metric + metric_name: faas.errors + brief: "Number of invocation errors" + instrument: counter + unit: "{error}" + attributes: + - ref: faas.trigger + + - id: metric.faas.invocations + type: metric + metric_name: faas.invocations + brief: "Number of successful invocations" + instrument: counter + unit: "{invocation}" + attributes: + - ref: faas.trigger + + - id: metric.faas.timeouts + type: metric + metric_name: faas.timeouts + brief: "Number of invocation timeouts" + instrument: counter + unit: "{timeout}" + attributes: + - ref: faas.trigger + + - id: metric.faas.mem_usage + type: metric + metric_name: faas.mem_usage + brief: "Distribution of max memory usage per invocation" + instrument: histogram + unit: "By" + attributes: + - ref: faas.trigger + + - id: metric.faas.cpu_usage + type: metric + metric_name: faas.cpu_usage + brief: "Distribution of CPU usage per invocation" + instrument: histogram + unit: "s" + attributes: + - ref: faas.trigger + + - id: metric.faas.net_io + type: metric + metric_name: faas.net_io + brief: "Distribution of net I/O usage per invocation" + instrument: histogram + unit: "By" + attributes: + - ref: faas.trigger diff --git a/internal/otelschema/_testdata/model/metrics/http.yaml b/internal/otelschema/_testdata/model/metrics/http.yaml new file mode 100644 index 00000000..9e12896e --- /dev/null +++ b/internal/otelschema/_testdata/model/metrics/http.yaml @@ -0,0 +1,119 @@ +groups: + - id: metric_attributes.http.server + type: attribute_group + brief: 'HTTP server attributes' + extends: attributes.http.server + attributes: + - ref: server.address + requirement_level: opt_in + note: | + See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). + > **Warning** + > Since this attribute is based on HTTP headers, opting in to it may allow an attacker + > to trigger cardinality limits, degrading the usefulness of the metric. + - ref: server.port + requirement_level: opt_in + note: | + See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). + > **Warning** + > Since this attribute is based on HTTP headers, opting in to it may allow an attacker + > to trigger cardinality limits, degrading the usefulness of the metric. + - id: metric_attributes.http.client + type: attribute_group + brief: 'HTTP client attributes' + extends: attributes.http.client + + - id: metric.http.server.request.duration + type: metric + metric_name: http.server.request.duration + brief: "Duration of HTTP server requests." + instrument: histogram + unit: "s" + extends: metric_attributes.http.server + + - id: metric.http.server.active_requests + type: metric + metric_name: http.server.active_requests + brief: "Number of active HTTP server requests." + instrument: updowncounter + unit: "{request}" + attributes: + - ref: http.request.method + requirement_level: required + - ref: url.scheme + requirement_level: required + examples: ["http", "https"] + - ref: server.address + requirement_level: opt_in + brief: > + Name of the local HTTP server that received the request. + note: | + See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). + > **Warning** + > Since this attribute is based on HTTP headers, opting in to it may allow an attacker + > to trigger cardinality limits, degrading the usefulness of the metric. + - ref: server.port + requirement_level: opt_in + brief: > + Port of the local HTTP server that received the request. + note: | + See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). + > **Warning** + > Since this attribute is based on HTTP headers, opting in to it may allow an attacker + > to trigger cardinality limits, degrading the usefulness of the metric. + + - id: metric.http.server.request.body.size + type: metric + metric_name: http.server.request.body.size + brief: "Size of HTTP server request bodies." + instrument: histogram + unit: "By" + note: > + The size of the request payload body in bytes. This is the number of bytes transferred excluding headers and + is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) + header. For requests using transport encoding, this should be the compressed size. + extends: metric_attributes.http.server + + - id: metric.http.server.response.body.size + type: metric + metric_name: http.server.response.body.size + brief: "Size of HTTP server response bodies." + instrument: histogram + unit: "By" + note: > + The size of the response payload body in bytes. This is the number of bytes transferred excluding headers and + is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) + header. For requests using transport encoding, this should be the compressed size. + extends: metric_attributes.http.server + + - id: metric.http.client.request.duration + type: metric + metric_name: http.client.request.duration + brief: "Duration of HTTP client requests." + instrument: histogram + unit: "s" + extends: metric_attributes.http.client + + - id: metric.http.client.request.body.size + type: metric + metric_name: http.client.request.body.size + brief: "Size of HTTP client request bodies." + instrument: histogram + unit: "By" + note: > + The size of the request payload body in bytes. This is the number of bytes transferred excluding headers and + is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) + header. For requests using transport encoding, this should be the compressed size. + extends: metric_attributes.http.client + + - id: metric.http.client.response.body.size + type: metric + metric_name: http.client.response.body.size + brief: "Size of HTTP client response bodies." + instrument: histogram + unit: "By" + note: > + The size of the response payload body in bytes. This is the number of bytes transferred excluding headers and + is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) + header. For requests using transport encoding, this should be the compressed size. + extends: metric_attributes.http.client diff --git a/internal/otelschema/_testdata/model/metrics/jvm-metrics-experimental.yaml b/internal/otelschema/_testdata/model/metrics/jvm-metrics-experimental.yaml new file mode 100644 index 00000000..8ea4fa57 --- /dev/null +++ b/internal/otelschema/_testdata/model/metrics/jvm-metrics-experimental.yaml @@ -0,0 +1,70 @@ +groups: + - id: metric.jvm.memory.init + type: metric + metric_name: jvm.memory.init + extends: attributes.jvm.memory + brief: "Measure of initial memory requested." + instrument: updowncounter + unit: "By" + + - id: metric.jvm.system.cpu.utilization + type: metric + metric_name: jvm.system.cpu.utilization + brief: "Recent CPU utilization for the whole system as reported by the JVM." + note: > + The value range is [0.0,1.0]. + This utilization is not defined as being for the specific interval since last measurement + (unlike `system.cpu.utilization`). + [Reference](https://docs.oracle.com/en/java/javase/17/docs/api/jdk.management/com/sun/management/OperatingSystemMXBean.html#getCpuLoad()). + instrument: gauge + unit: "1" + + - id: metric.jvm.system.cpu.load_1m + type: metric + metric_name: jvm.system.cpu.load_1m + brief: "Average CPU load of the whole system for the last minute as reported by the JVM." + note: > + The value range is [0,n], where n is the number of CPU cores - or a negative number if the value is not available. + This utilization is not defined as being for the specific interval since last measurement + (unlike `system.cpu.utilization`). + [Reference](https://docs.oracle.com/en/java/javase/17/docs/api/java.management/java/lang/management/OperatingSystemMXBean.html#getSystemLoadAverage()). + instrument: gauge + unit: "{run_queue_item}" + + - id: attributes.jvm.buffer + type: attribute_group + brief: "Describes JVM buffer metric attributes." + prefix: jvm.buffer + attributes: + - id: pool.name + type: string + requirement_level: recommended + brief: Name of the buffer pool. + examples: [ "mapped", "direct" ] + note: > + Pool names are generally obtained via + [BufferPoolMXBean#getName()](https://docs.oracle.com/en/java/javase/11/docs/api/java.management/java/lang/management/BufferPoolMXBean.html#getName()). + + - id: metric.jvm.buffer.memory.usage + type: metric + metric_name: jvm.buffer.memory.usage + extends: attributes.jvm.buffer + brief: "Measure of memory used by buffers." + instrument: updowncounter + unit: "By" + + - id: metric.jvm.buffer.memory.limit + type: metric + metric_name: jvm.buffer.memory.limit + extends: attributes.jvm.buffer + brief: "Measure of total memory capacity of buffers." + instrument: updowncounter + unit: "By" + + - id: metric.jvm.buffer.count + type: metric + metric_name: jvm.buffer.count + extends: attributes.jvm.buffer + brief: "Number of buffers in the pool." + instrument: updowncounter + unit: "{buffer}" diff --git a/internal/otelschema/_testdata/model/metrics/jvm-metrics.yaml b/internal/otelschema/_testdata/model/metrics/jvm-metrics.yaml new file mode 100644 index 00000000..ba1f5604 --- /dev/null +++ b/internal/otelschema/_testdata/model/metrics/jvm-metrics.yaml @@ -0,0 +1,168 @@ +groups: + - id: attributes.jvm.memory + type: attribute_group + brief: "Describes JVM memory metric attributes." + prefix: jvm.memory + attributes: + - id: type + type: + allow_custom_values: false + members: + - id: heap + value: 'heap' + brief: 'Heap memory.' + - id: non_heap + value: 'non_heap' + brief: 'Non-heap memory' + requirement_level: recommended + brief: The type of memory. + examples: ["heap", "non_heap"] + - id: pool.name + type: string + requirement_level: recommended + brief: Name of the memory pool. + examples: ["G1 Old Gen", "G1 Eden space", "G1 Survivor Space"] + note: > + Pool names are generally obtained via + [MemoryPoolMXBean#getName()](https://docs.oracle.com/en/java/javase/11/docs/api/java.management/java/lang/management/MemoryPoolMXBean.html#getName()). + + - id: metric.jvm.memory.used + type: metric + metric_name: jvm.memory.used + extends: attributes.jvm.memory + brief: "Measure of memory used." + instrument: updowncounter + unit: "By" + + - id: metric.jvm.memory.committed + type: metric + metric_name: jvm.memory.committed + extends: attributes.jvm.memory + brief: "Measure of memory committed." + instrument: updowncounter + unit: "By" + + - id: metric.jvm.memory.limit + type: metric + metric_name: jvm.memory.limit + extends: attributes.jvm.memory + brief: "Measure of max obtainable memory." + instrument: updowncounter + unit: "By" + + - id: metric.jvm.memory.used_after_last_gc + type: metric + metric_name: jvm.memory.used_after_last_gc + extends: attributes.jvm.memory + brief: "Measure of memory used, as measured after the most recent garbage collection event on this pool." + instrument: updowncounter + unit: "By" + + - id: metric.jvm.gc.duration + type: metric + metric_name: jvm.gc.duration + brief: "Duration of JVM garbage collection actions." + instrument: histogram + unit: "s" + prefix: jvm.gc + attributes: + - id: name + type: string + requirement_level: recommended + brief: Name of the garbage collector. + examples: ["G1 Young Generation", "G1 Old Generation"] + note: > + Garbage collector name is generally obtained via + [GarbageCollectionNotificationInfo#getGcName()](https://docs.oracle.com/en/java/javase/11/docs/api/jdk.management/com/sun/management/GarbageCollectionNotificationInfo.html#getGcName()). + - id: action + type: string + requirement_level: recommended + brief: Name of the garbage collector action. + examples: ["end of minor GC", "end of major GC"] + note: > + Garbage collector action is generally obtained via + [GarbageCollectionNotificationInfo#getGcAction()](https://docs.oracle.com/en/java/javase/11/docs/api/jdk.management/com/sun/management/GarbageCollectionNotificationInfo.html#getGcAction()). + + - id: metric.jvm.thread.count + type: metric + metric_name: jvm.thread.count + brief: "Number of executing platform threads." + instrument: updowncounter + unit: "{thread}" + attributes: + - id: jvm.thread.daemon + type: boolean + requirement_level: recommended + brief: "Whether the thread is daemon or not." + - id: jvm.thread.state + requirement_level: recommended + type: + allow_custom_values: false + members: + - id: new + value: 'new' + brief: 'A thread that has not yet started is in this state.' + - id: runnable + value: 'runnable' + brief: 'A thread executing in the Java virtual machine is in this state.' + - id: blocked + value: 'blocked' + brief: 'A thread that is blocked waiting for a monitor lock is in this state.' + - id: waiting + value: 'waiting' + brief: 'A thread that is waiting indefinitely for another thread to perform a particular action is in this state.' + - id: timed_waiting + value: 'timed_waiting' + brief: 'A thread that is waiting for another thread to perform an action for up to a specified waiting time is in this state.' + - id: terminated + value: 'terminated' + brief: 'A thread that has exited is in this state.' + brief: "State of the thread." + examples: ["runnable", "blocked"] + + - id: metric.jvm.class.loaded + type: metric + metric_name: jvm.class.loaded + brief: "Number of classes loaded since JVM start." + instrument: counter + unit: "{class}" + + - id: metric.jvm.class.unloaded + type: metric + metric_name: jvm.class.unloaded + brief: "Number of classes unloaded since JVM start." + instrument: counter + unit: "{class}" + + - id: metric.jvm.class.count + type: metric + metric_name: jvm.class.count + brief: "Number of classes currently loaded." + instrument: updowncounter + unit: "{class}" + + - id: metric.jvm.cpu.count + type: metric + metric_name: jvm.cpu.count + brief: "Number of processors available to the Java virtual machine." + instrument: updowncounter + unit: "{cpu}" + + - id: metric.jvm.cpu.time + type: metric + metric_name: jvm.cpu.time + brief: "CPU time used by the process as reported by the JVM." + instrument: counter + unit: "s" + + - id: metric.jvm.cpu.recent_utilization + type: metric + metric_name: jvm.cpu.recent_utilization + brief: "Recent CPU utilization for the process as reported by the JVM." + note: > + The value range is [0.0,1.0]. + This utilization is not defined as being for the specific interval since last measurement + (unlike `system.cpu.utilization`). + [Reference](https://docs.oracle.com/en/java/javase/17/docs/api/jdk.management/com/sun/management/OperatingSystemMXBean.html#getProcessCpuLoad()). + instrument: gauge + unit: "1" diff --git a/internal/otelschema/_testdata/model/metrics/rpc-metrics.yaml b/internal/otelschema/_testdata/model/metrics/rpc-metrics.yaml new file mode 100644 index 00000000..ca29cdf1 --- /dev/null +++ b/internal/otelschema/_testdata/model/metrics/rpc-metrics.yaml @@ -0,0 +1,122 @@ +groups: + # TODO: Should we list the attributes on each metric + # OR leave a single table like it is today? Since all attributes are applied + # to all metrics, the repetition of them bloats the page + - id: attributes.metrics.rpc + type: attribute_group + brief: "Describes RPC metric attributes." + attributes: + - ref: rpc.system + requirement_level: required + - ref: rpc.service + - ref: rpc.method + - ref: network.transport + - ref: network.type + - ref: server.address + - ref: server.port + + # RPC Server metrics + - id: metric.rpc.server.duration + type: metric + metric_name: rpc.server.duration + brief: Measures the duration of inbound RPC. + instrument: histogram + unit: "ms" + note: | + While streaming RPCs may record this metric as start-of-batch + to end-of-batch, it's hard to interpret in practice. + + **Streaming**: N/A. + + - id: metric.rpc.server.request.size + type: metric + metric_name: rpc.server.request.size + brief: Measures the size of RPC request messages (uncompressed). + instrument: histogram + unit: "By" + note: | + **Streaming**: Recorded per message in a streaming batch + + - id: metric.rpc.server.response.size + type: metric + metric_name: rpc.server.response.size + brief: Measures the size of RPC response messages (uncompressed). + instrument: histogram + unit: "By" + note: | + **Streaming**: Recorded per response in a streaming batch + + - id: metric.rpc.server.requests_per_rpc + type: metric + metric_name: rpc.server.requests_per_rpc + brief: Measures the number of messages received per RPC. + instrument: histogram + unit: "{count}" + note: | + Should be 1 for all non-streaming RPCs. + + **Streaming** : This metric is required for server and client streaming RPCs + + - id: metric.rpc.server.responses_per_rpc + type: metric + metric_name: rpc.server.responses_per_rpc + brief: Measures the number of messages sent per RPC. + instrument: histogram + unit: "{count}" + note: | + Should be 1 for all non-streaming RPCs. + + **Streaming**: This metric is required for server and client streaming RPCs + + # RPC Client metrics + - id: metric.rpc.client.duration + type: metric + metric_name: rpc.client.duration + brief: Measures the duration of outbound RPC. + instrument: histogram + unit: "ms" + note: | + While streaming RPCs may record this metric as start-of-batch + to end-of-batch, it's hard to interpret in practice. + + **Streaming**: N/A. + + - id: metric.rpc.client.request.size + type: metric + metric_name: rpc.client.request.size + brief: Measures the size of RPC request messages (uncompressed). + instrument: histogram + unit: "By" + note: | + **Streaming**: Recorded per message in a streaming batch + + - id: metric.rpc.client.response.size + type: metric + metric_name: rpc.client.response.size + brief: Measures the size of RPC response messages (uncompressed). + instrument: histogram + unit: "By" + note: | + **Streaming**: Recorded per response in a streaming batch + + - id: metric.rpc.client.requests_per_rpc + type: metric + metric_name: rpc.client.requests_per_rpc + brief: Measures the number of messages received per RPC. + instrument: histogram + unit: "{count}" + note: | + Should be 1 for all non-streaming RPCs. + + **Streaming**: This metric is required for server and client streaming RPCs + + - id: metric.rpc.client.responses_per_rpc + type: metric + metric_name: rpc.client.responses_per_rpc + brief: Measures the number of messages sent per RPC. + instrument: histogram + unit: "{count}" + note: | + Should be 1 for all non-streaming RPCs. + + **Streaming**: This metric is required for server and client streaming RPCs diff --git a/internal/otelschema/_testdata/model/metrics/system-metrics.yaml b/internal/otelschema/_testdata/model/metrics/system-metrics.yaml new file mode 100644 index 00000000..d411115e --- /dev/null +++ b/internal/otelschema/_testdata/model/metrics/system-metrics.yaml @@ -0,0 +1,501 @@ +groups: + # General system attributes + - id: attributes.system + prefix: system + type: attribute_group + brief: "Describes System metric attributes" + attributes: + - id: device + type: string + brief: "The device identifier" + examples: ["(identifier)"] + + # system.cpu.* metrics and attribute group + - id: attributes.system.cpu + prefix: system.cpu + type: attribute_group + brief: "Describes System CPU metric attributes" + attributes: + - id: state + type: + allow_custom_values: true + members: + - id: user + value: 'user' + - id: system + value: 'system' + - id: nice + value: 'nice' + - id: idle + value: 'idle' + - id: iowait + value: 'iowait' + - id: interrupt + value: 'interrupt' + - id: steal + value: 'steal' + brief: "The state of the CPU" + examples: ["idle", "interrupt"] + - id: logical_number + type: int + brief: "The logical CPU number [0..n-1]" + examples: [1] + + - id: metric.system.cpu.time + type: metric + metric_name: system.cpu.time + brief: "Seconds each logical CPU spent on each mode" + instrument: counter + unit: "s" + attributes: + - ref: system.cpu.state + - ref: system.cpu.logical_number + + - id: metric.system.cpu.utilization + type: metric + metric_name: system.cpu.utilization + brief: "Difference in system.cpu.time since the last measurement, divided by the elapsed time and number of logical CPUs" + instrument: gauge + unit: "1" + attributes: + - ref: system.cpu.state + - ref: system.cpu.logical_number + + - id: metric.system.cpu.frequency + type: metric + metric_name: system.cpu.frequency + brief: "Reports the current frequency of the CPU in Hz" + instrument: gauge + unit: "{Hz}" + attributes: + - ref: system.cpu.logical_number + + - id: metric.system.cpu.physical.count + type: metric + metric_name: system.cpu.physical.count + brief: "Reports the number of actual physical processor cores on the hardware" + instrument: updowncounter + unit: "{cpu}" + attributes: [] + + - id: metric.system.cpu.logical.count + type: metric + metric_name: system.cpu.logical.count + brief: "Reports the number of logical (virtual) processor cores created by the operating system to manage multitasking" + instrument: updowncounter + unit: "{cpu}" + attributes: [] + + # sytem.memory.* metrics and attribute group + - id: attributes.system.memory + prefix: system.memory + type: attribute_group + brief: "Describes System Memory metric attributes" + attributes: + - id: state + type: + allow_custom_values: true + members: + - id: used + value: 'used' + - id: free + value: 'free' + - id: shared + value: 'shared' + - id: buffers + value: 'buffers' + - id: cached + value: 'cached' + brief: "The memory state" + examples: ["free", "cached"] + + - id: metric.system.memory.usage + type: metric + metric_name: system.memory.usage + brief: "Reports memory in use by state." + note: | + The sum over all `system.memory.state` values SHOULD equal the total memory + available on the system, that is `system.memory.limit`. + instrument: updowncounter + unit: "By" + attributes: + - ref: system.memory.state + + - id: metric.system.memory.limit + type: metric + metric_name: system.memory.limit + brief: "Total memory available in the system." + note: | + Its value SHOULD equal the sum of `system.memory.state` over all states. + instrument: updowncounter + unit: "By" + + - id: metric.system.memory.utilization + type: metric + metric_name: system.memory.utilization + brief: "" + instrument: gauge + unit: "1" + attributes: + - ref: system.memory.state + + # system.paging.* metrics and attribute group + - id: attributes.system.paging + prefix: system.paging + type: attribute_group + brief: "Describes System Memory Paging metric attributes" + attributes: + - id: state + type: + allow_custom_values: false + members: + - id: used + value: 'used' + - id: free + value: 'free' + brief: "The memory paging state" + examples: ["free"] + - id: type + type: + allow_custom_values: false + members: + - id: major + value: 'major' + - id: minor + value: 'minor' + brief: "The memory paging type" + examples: ["minor"] + - id: direction + type: + allow_custom_values: false + members: + - id: in + value: 'in' + - id: out + value: 'out' + brief: "The paging access direction" + examples: ["in"] + - id: metric.system.paging.usage + type: metric + metric_name: system.paging.usage + brief: "Unix swap or windows pagefile usage" + instrument: updowncounter + unit: "By" + attributes: + - ref: system.paging.state + + - id: metric.system.paging.utilization + type: metric + metric_name: system.paging.utilization + brief: "" + instrument: gauge + unit: "1" + attributes: + - ref: system.paging.state + + - id: metric.system.paging.faults + type: metric + metric_name: system.paging.faults + brief: "" + instrument: counter + unit: "{fault}" + attributes: + - ref: system.paging.type + + - id: metric.system.paging.operations + type: metric + metric_name: system.paging.operations + brief: "" + instrument: counter + unit: "{operation}" + attributes: + - ref: system.paging.type + - ref: system.paging.direction + + # system.disk.* metrics and attribute group + - id: metric.system.disk.io + type: metric + metric_name: system.disk.io + brief: "" + instrument: counter + unit: "By" + attributes: + - ref: system.device + - ref: disk.io.direction + + - id: metric.system.disk.operations + type: metric + metric_name: system.disk.operations + brief: "" + instrument: counter + unit: "{operation}" + attributes: + - ref: system.device + - ref: disk.io.direction + + - id: metric.system.disk.io_time + type: metric + metric_name: system.disk.io_time + brief: "Time disk spent activated" + instrument: counter + unit: "s" + note: | + The real elapsed time ("wall clock") used in the I/O path (time from operations running in parallel are not counted). Measured as: + + - Linux: Field 13 from [procfs-diskstats](https://www.kernel.org/doc/Documentation/ABI/testing/procfs-diskstats) + - Windows: The complement of + ["Disk\% Idle Time"](https://learn.microsoft.com/archive/blogs/askcore/windows-performance-monitor-disk-counters-explained#windows-performance-monitor-disk-counters-explained) + performance counter: `uptime * (100 - "Disk\% Idle Time") / 100` + attributes: + - ref: system.device + + - id: metric.system.disk.operation_time + type: metric + metric_name: system.disk.operation_time + brief: "Sum of the time each operation took to complete" + instrument: counter + unit: "s" + note: | + Because it is the sum of time each request took, parallel-issued requests each contribute to make the count grow. Measured as: + + - Linux: Fields 7 & 11 from [procfs-diskstats](https://www.kernel.org/doc/Documentation/ABI/testing/procfs-diskstats) + - Windows: "Avg. Disk sec/Read" perf counter multiplied by "Disk Reads/sec" perf counter (similar for Writes) + attributes: + - ref: system.device + - ref: disk.io.direction + + - id: metric.system.disk.merged + type: metric + metric_name: system.disk.merged + brief: "" + instrument: counter + unit: "{operation}" + attributes: + - ref: system.device + - ref: disk.io.direction + + # system.filesystem.* metrics and attribute group + - id: attributes.system.filesystem + prefix: system.filesystem + type: attribute_group + brief: "Describes Filesystem metric attributes" + attributes: + - id: state + brief: "The filesystem state" + type: + allow_custom_values: false + members: + - id: used + value: 'used' + - id: free + value: 'free' + - id: reserved + value: 'reserved' + examples: ["used"] + - id: type + type: + allow_custom_values: true + members: + - id: fat32 + value: 'fat32' + - id: exfat + value: 'exfat' + - id: ntfs + value: 'ntfs' + - id: refs + value: 'refs' + - id: hfsplus + value: 'hfsplus' + - id: ext4 + value: 'ext4' + brief: "The filesystem type" + examples: ["ext4"] + - id: mode + type: string + brief: "The filesystem mode" + examples: ["rw, ro"] + - id: mountpoint + type: string + brief: "The filesystem mount path" + examples: ["/mnt/data"] + + - id: metric.system.filesystem.usage + type: metric + metric_name: system.filesystem.usage + brief: "" + instrument: updowncounter + unit: "By" + attributes: + - ref: system.device + - ref: system.filesystem.state + - ref: system.filesystem.type + - ref: system.filesystem.mode + - ref: system.filesystem.mountpoint + + - id: metric.system.filesystem.utilization + type: metric + metric_name: system.filesystem.utilization + brief: "" + instrument: gauge + unit: "1" + attributes: + - ref: system.device + - ref: system.filesystem.state + - ref: system.filesystem.type + - ref: system.filesystem.mode + - ref: system.filesystem.mountpoint + + # system.network.* metrics and attribute group + + # System-specific network attributes + - id: attributes.system.network + prefix: system.network + type: attribute_group + brief: "Describes Network metric attributes" + attributes: + - id: state + type: + allow_custom_values: false + members: + - id: close + value: 'close' + - id: close_wait + value: 'close_wait' + - id: closing + value: 'closing' + - id: delete + value: 'delete' + - id: established + value: 'established' + - id: fin_wait_1 + value: 'fin_wait_1' + - id: fin_wait_2 + value: 'fin_wait_2' + - id: last_ack + value: 'last_ack' + - id: listen + value: 'listen' + - id: syn_recv + value: 'syn_recv' + - id: syn_sent + value: 'syn_sent' + - id: time_wait + value: 'time_wait' + brief: "A stateless protocol MUST NOT set this attribute" + examples: ["close_wait"] + + - id: metric.system.network.dropped + type: metric + metric_name: system.network.dropped + brief: "Count of packets that are dropped or discarded even though there was no error" + instrument: counter + unit: "{packet}" + note: | + Measured as: + + - Linux: the `drop` column in `/proc/dev/net` ([source](https://web.archive.org/web/20180321091318/http://www.onlamp.com/pub/a/linux/2000/11/16/LinuxAdmin.html)) + - Windows: [`InDiscards`/`OutDiscards`](https://docs.microsoft.com/windows/win32/api/netioapi/ns-netioapi-mib_if_row2) + from [`GetIfEntry2`](https://docs.microsoft.com/windows/win32/api/netioapi/nf-netioapi-getifentry2) + attributes: + - ref: system.device + - ref: network.io.direction + + - id: metric.system.network.packets + type: metric + metric_name: system.network.packets + brief: "" + instrument: counter + unit: "{packet}" + attributes: + - ref: system.device + - ref: network.io.direction + + - id: metric.system.network.errors + type: metric + metric_name: system.network.errors + brief: "Count of network errors detected" + instrument: counter + unit: "{error}" + note: | + Measured as: + + - Linux: the `errs` column in `/proc/dev/net` ([source](https://web.archive.org/web/20180321091318/http://www.onlamp.com/pub/a/linux/2000/11/16/LinuxAdmin.html)). + - Windows: [`InErrors`/`OutErrors`](https://docs.microsoft.com/windows/win32/api/netioapi/ns-netioapi-mib_if_row2) + from [`GetIfEntry2`](https://docs.microsoft.com/windows/win32/api/netioapi/nf-netioapi-getifentry2). + attributes: + - ref: system.device + - ref: network.io.direction + + - id: metric.system.network.io + type: metric + metric_name: system.network.io + brief: "" + instrument: counter + unit: "By" + attributes: + - ref: system.device + - ref: network.io.direction + + - id: metric.system.network.connections + type: metric + metric_name: system.network.connections + brief: "" + instrument: updowncounter + unit: "{connection}" + attributes: + - ref: system.device + - ref: system.network.state + - ref: network.transport + + # system.processes.* metrics and attribute group + - id: attributes.system.processes + prefix: system.processes + type: attribute_group + brief: "Describes System Process metric attributes" + attributes: + - id: status + type: + allow_custom_values: true + members: + - id: running + value: 'running' + - id: sleeping + value: 'sleeping' + - id: stopped + value: 'stopped' + - id: defunct + value: 'defunct' + brief: > + The process state, e.g., [Linux Process State Codes](https://man7.org/linux/man-pages/man1/ps.1.html#PROCESS_STATE_CODES) + examples: ["running"] + + + - id: metric.system.processes.count + type: metric + metric_name: system.processes.count + brief: "Total number of processes in each state" + instrument: updowncounter + unit: "{process}" + attributes: + - ref: system.processes.status + + - id: metric.system.processes.created + type: metric + metric_name: system.processes.created + brief: "Total number of processes created over uptime of the host" + instrument: counter + unit: "{process}" + + # system.linux.* metrics + - id: metric.system.linux.memory.available + type: metric + metric_name: system.linux.memory.available + brief: "An estimate of how much memory is available for starting new applications, without causing swapping" + note: | + This is an alternative to `system.memory.usage` metric with `state=free`. + Linux starting from 3.14 exports "available" memory. It takes "free" memory as a baseline, and then factors in kernel-specific values. + This is supposed to be more accurate than just "free" memory. + For reference, see the calculations [here](https://superuser.com/a/980821). + See also `MemAvailable` in [/proc/meminfo](https://man7.org/linux/man-pages/man5/proc.5.html). + instrument: updowncounter + unit: "By" diff --git a/internal/otelschema/_testdata/model/network.yaml b/internal/otelschema/_testdata/model/network.yaml new file mode 100644 index 00000000..7e1ee396 --- /dev/null +++ b/internal/otelschema/_testdata/model/network.yaml @@ -0,0 +1,28 @@ +groups: + - id: network-core + prefix: network + type: attribute_group + brief: > + These attributes may be used for any network related operation. + attributes: + - ref: network.transport + - ref: network.type + - ref: network.protocol.name + - ref: network.protocol.version + - ref: network.peer.address + - ref: network.peer.port + - ref: network.local.address + - ref: network.local.port + + - id: network-connection-and-carrier + prefix: network + type: attribute_group + brief: > + These attributes may be used for any network related operation. + attributes: + - ref: network.connection.type + - ref: network.connection.subtype + - ref: network.carrier.name + - ref: network.carrier.mcc + - ref: network.carrier.mnc + - ref: network.carrier.icc diff --git a/internal/otelschema/_testdata/model/registry/client.yaml b/internal/otelschema/_testdata/model/registry/client.yaml new file mode 100644 index 00000000..bd214a17 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/client.yaml @@ -0,0 +1,28 @@ +groups: + - id: client + prefix: client + type: attribute_group + brief: > + These attributes may be used to describe the client in a connection-based network interaction + where there is one side that initiates the connection (the client is the side that initiates the connection). + This covers all TCP network interactions since TCP is connection-based and one side initiates the + connection (an exception is made for peer-to-peer communication over TCP where the "user-facing" surface of the + protocol / API doesn't expose a clear notion of client and server). + This also covers UDP network interactions where one side initiates the interaction, e.g. QUIC (HTTP/3) and DNS. + attributes: + - id: address + stability: stable + type: string + brief: "Client address - domain name if available without reverse DNS lookup; otherwise, IP address or Unix domain socket name." + note: > + When observed from the server side, and when communicating through an intermediary, `client.address` SHOULD represent + the client address behind any intermediaries, for example proxies, if it's available. + examples: ['client.example.com', '10.1.2.80', '/tmp/my.sock'] + - id: port + stability: stable + type: int + brief: Client port number. + examples: [65123] + note: > + When observed from the server side, and when communicating through an intermediary, `client.port` SHOULD represent + the client port behind any intermediaries, for example proxies, if it's available. diff --git a/internal/otelschema/_testdata/model/registry/cloud.yaml b/internal/otelschema/_testdata/model/registry/cloud.yaml new file mode 100644 index 00000000..2c669052 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/cloud.yaml @@ -0,0 +1,179 @@ +groups: + - id: registry.cloud + prefix: cloud + type: resource + brief: > + A cloud environment (e.g. GCP, Azure, AWS). + attributes: + - id: provider + type: + allow_custom_values: true + members: + - id: 'alibaba_cloud' + value: 'alibaba_cloud' + brief: 'Alibaba Cloud' + - id: 'aws' + value: 'aws' + brief: 'Amazon Web Services' + - id: 'azure' + value: 'azure' + brief: 'Microsoft Azure' + - id: 'gcp' + value: 'gcp' + brief: 'Google Cloud Platform' + - id: 'heroku' + value: 'heroku' + brief: 'Heroku Platform as a Service' + - id: 'ibm_cloud' + value: 'ibm_cloud' + brief: 'IBM Cloud' + - id: 'tencent_cloud' + value: 'tencent_cloud' + brief: 'Tencent Cloud' + + brief: > + Name of the cloud provider. + - id: account.id + type: string + brief: > + The cloud account ID the resource is assigned to. + examples: ['111111111111', 'opentelemetry'] + - id: region + type: string + brief: > + The geographical region the resource is running. + note: > + Refer to your provider's docs to see the available regions, for example + [Alibaba Cloud regions](https://www.alibabacloud.com/help/doc-detail/40654.htm), + [AWS regions](https://aws.amazon.com/about-aws/global-infrastructure/regions_az/), + [Azure regions](https://azure.microsoft.com/global-infrastructure/geographies/), + [Google Cloud regions](https://cloud.google.com/about/locations), + or [Tencent Cloud regions](https://www.tencentcloud.com/document/product/213/6091). + examples: ['us-central1', 'us-east-1'] + - id: resource_id + type: string + brief: > + Cloud provider-specific native identifier of the monitored cloud resource + (e.g. an [ARN](https://docs.aws.amazon.com/general/latest/gr/aws-arns-and-namespaces.html) on AWS, + a [fully qualified resource ID](https://learn.microsoft.com/rest/api/resources/resources/get-by-id) on Azure, + a [full resource name](https://cloud.google.com/apis/design/resource_names#full_resource_name) on GCP) + note: | + On some cloud providers, it may not be possible to determine the full ID at startup, + so it may be necessary to set `cloud.resource_id` as a span attribute instead. + + The exact value to use for `cloud.resource_id` depends on the cloud provider. + The following well-known definitions MUST be used if you set this attribute and they apply: + + * **AWS Lambda:** The function [ARN](https://docs.aws.amazon.com/general/latest/gr/aws-arns-and-namespaces.html). + Take care not to use the "invoked ARN" directly but replace any + [alias suffix](https://docs.aws.amazon.com/lambda/latest/dg/configuration-aliases.html) + with the resolved function version, as the same runtime instance may be invokable with + multiple different aliases. + * **GCP:** The [URI of the resource](https://cloud.google.com/iam/docs/full-resource-names) + * **Azure:** The [Fully Qualified Resource ID](https://docs.microsoft.com/rest/api/resources/resources/get-by-id) of the invoked function, + *not* the function app, having the form + `/subscriptions//resourceGroups//providers/Microsoft.Web/sites//functions/`. + This means that a span attribute MUST be used, as an Azure function app can host multiple functions that would usually share + a TracerProvider. + examples: + - 'arn:aws:lambda:REGION:ACCOUNT_ID:function:my-function' + - '//run.googleapis.com/projects/PROJECT_ID/locations/LOCATION_ID/services/SERVICE_ID' + - '/subscriptions//resourceGroups//providers/Microsoft.Web/sites//functions/' + - id: availability_zone + type: string + brief: > + Cloud regions often have multiple, isolated locations known as zones + to increase availability. Availability zone represents the + zone where the resource is running. + note: > + Availability zones are called "zones" on Alibaba Cloud and Google Cloud. + examples: ['us-east-1c'] + - id: platform + type: + allow_custom_values: true + members: + - id: alibaba_cloud_ecs + value: 'alibaba_cloud_ecs' + brief: Alibaba Cloud Elastic Compute Service + - id: alibaba_cloud_fc + value: 'alibaba_cloud_fc' + brief: Alibaba Cloud Function Compute + - id: alibaba_cloud_openshift + value: 'alibaba_cloud_openshift' + brief: Red Hat OpenShift on Alibaba Cloud + - id: aws_ec2 + value: 'aws_ec2' + brief: AWS Elastic Compute Cloud + - id: aws_ecs + value: 'aws_ecs' + brief: AWS Elastic Container Service + - id: aws_eks + value: 'aws_eks' + brief: AWS Elastic Kubernetes Service + - id: aws_lambda + value: 'aws_lambda' + brief: AWS Lambda + - id: aws_elastic_beanstalk + value: 'aws_elastic_beanstalk' + brief: AWS Elastic Beanstalk + - id: aws_app_runner + value: 'aws_app_runner' + brief: AWS App Runner + - id: aws_openshift + value: 'aws_openshift' + brief: Red Hat OpenShift on AWS (ROSA) + - id: azure_vm + value: 'azure_vm' + brief: Azure Virtual Machines + - id: azure_container_instances + value: 'azure_container_instances' + brief: Azure Container Instances + - id: azure_aks + value: 'azure_aks' + brief: Azure Kubernetes Service + - id: azure_functions + value: 'azure_functions' + brief: Azure Functions + - id: azure_app_service + value: 'azure_app_service' + brief: Azure App Service + - id: azure_openshift + value: 'azure_openshift' + brief: Azure Red Hat OpenShift + - id: gcp_bare_metal_solution + value: 'gcp_bare_metal_solution' + brief: Google Bare Metal Solution (BMS) + - id: gcp_compute_engine + value: 'gcp_compute_engine' + brief: Google Cloud Compute Engine (GCE) + - id: gcp_cloud_run + value: 'gcp_cloud_run' + brief: Google Cloud Run + - id: gcp_kubernetes_engine + value: 'gcp_kubernetes_engine' + brief: Google Cloud Kubernetes Engine (GKE) + - id: gcp_cloud_functions + value: 'gcp_cloud_functions' + brief: Google Cloud Functions (GCF) + - id: gcp_app_engine + value: 'gcp_app_engine' + brief: Google Cloud App Engine (GAE) + - id: gcp_openshift + value: 'gcp_openshift' + brief: Red Hat OpenShift on Google Cloud + - id: ibm_cloud_openshift + value: 'ibm_cloud_openshift' + brief: Red Hat OpenShift on IBM Cloud + - id: tencent_cloud_cvm + value: 'tencent_cloud_cvm' + brief: Tencent Cloud Cloud Virtual Machine (CVM) + - id: tencent_cloud_eks + value: 'tencent_cloud_eks' + brief: Tencent Cloud Elastic Kubernetes Service (EKS) + - id: tencent_cloud_scf + value: 'tencent_cloud_scf' + brief: Tencent Cloud Serverless Cloud Function (SCF) + brief: > + The cloud platform in use. + note: > + The prefix of the service SHOULD match the one specified in `cloud.provider`. diff --git a/internal/otelschema/_testdata/model/registry/code.yaml b/internal/otelschema/_testdata/model/registry/code.yaml new file mode 100644 index 00000000..d5afceea --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/code.yaml @@ -0,0 +1,41 @@ +groups: + - id: registry.code + prefix: code + type: span + brief: > + These attributes allow to report this unit of code and therefore to provide more context about the span. + attributes: + - id: function + type: string + brief: > + The method or function name, or equivalent (usually rightmost part of the code unit's name). + examples: serveRequest + - id: namespace + type: string + brief: > + The "namespace" within which `code.function` is defined. Usually the qualified class or module name, + such that `code.namespace` + some separator + `code.function` form a unique identifier for the code unit. + examples: com.example.MyHttpService + - id: filepath + type: string + brief: > + The source code file name that identifies the code unit as uniquely as possible (preferably an absolute file path). + examples: /usr/local/MyApplication/content_root/app/index.php + - id: lineno + type: int + brief: > + The line number in `code.filepath` best representing the operation. It SHOULD point within the code unit named in `code.function`. + examples: 42 + - id: column + type: int + brief: > + The column number in `code.filepath` best representing the operation. It SHOULD point within the code unit named in `code.function`. + examples: 16 + - id: stacktrace + type: string + brief: > + A stacktrace as a string in the natural representation for the language runtime. + The representation is to be determined and documented by each language SIG. + examples: 'at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n + at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n + at com.example.GenerateTrace.main(GenerateTrace.java:5)' diff --git a/internal/otelschema/_testdata/model/registry/container.yaml b/internal/otelschema/_testdata/model/registry/container.yaml new file mode 100644 index 00000000..0755d078 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/container.yaml @@ -0,0 +1,86 @@ +groups: + - id: registry.container + prefix: container + type: resource + brief: > + A container instance. + attributes: + - id: name + type: string + brief: > + Container name used by container runtime. + examples: ['opentelemetry-autoconf'] + - id: id + type: string + brief: > + Container ID. Usually a UUID, as for example used to + [identify Docker containers](https://docs.docker.com/engine/reference/run/#container-identification). + The UUID might be abbreviated. + examples: ['a3bf90e006b2'] + - id: runtime + type: string + brief: > + The container runtime managing this container. + examples: ['docker', 'containerd', 'rkt'] + - id: image.name + type: string + brief: > + Name of the image the container was built on. + examples: ['gcr.io/opentelemetry/operator'] + - id: image.tags + type: string[] + brief: > + Container image tags. An example can be found in + [Docker Image Inspect](https://docs.docker.com/engine/api/v1.43/#tag/Image/operation/ImageInspect). + Should be only the `` section of the full name for example + from `registry.example.com/my-org/my-image:`. + examples: ['v1.27.1', '3.5.7-0'] + - id: image.id + type: string + brief: > + Runtime specific image identifier. Usually a hash algorithm followed by a UUID. + note: > + Docker defines a sha256 of the image id; `container.image.id` corresponds to the `Image` field from the Docker + container inspect [API](https://docs.docker.com/engine/api/v1.43/#tag/Container/operation/ContainerInspect) + endpoint. + + K8s defines a link to the container registry repository with digest `"imageID": "registry.azurecr.io + /namespace/service/dockerfile@sha256:bdeabd40c3a8a492eaf9e8e44d0ebbb84bac7ee25ac0cf8a7159d25f62555625"`. + + The ID is assinged by the container runtime and can vary in different environments. + Consider using `oci.manifest.digest` if it is important to identify the same + image in different environments/runtimes. + examples: ['sha256:19c92d0a00d1b66d897bceaa7319bee0dd38a10a851c60bcec9474aa3f01e50f'] + - id: image.repo_digests + type: string[] + brief: > + Repo digests of the container image as provided by the container runtime. + note: > + [Docker](https://docs.docker.com/engine/api/v1.43/#tag/Image/operation/ImageInspect) and + [CRI](https://github.com/kubernetes/cri-api/blob/c75ef5b473bbe2d0a4fc92f82235efd665ea8e9f/pkg/apis/runtime/v1/api.proto#L1237-L1238) + report those under the `RepoDigests` field. + examples: + - 'example@sha256:afcc7f1ac1b49db317a7196c902e61c6c3c4607d63599ee1a82d702d249a0ccb' + - 'internal.registry.example.com:5000/example@sha256:b69959407d21e8a062e0416bf13405bb2b71ed7a84dde4158ebafacfa06f5578' + - id: command + type: string + note: > + If using embedded credentials or sensitive data, it is recommended to remove them to prevent potential leakage. + brief: > + The command used to run the container (i.e. the command name). + examples: [ 'otelcontribcol' ] + - id: command_line + type: string + brief: > + The full command run by the container as a single string representing the full command. [2] + examples: [ 'otelcontribcol --config config.yaml' ] + - id: command_args + type: string[] + brief: > + All the command arguments (including the command/executable itself) run by the container. [2] + examples: [ 'otelcontribcol, --config, config.yaml' ] + - id: labels + type: template[string] + brief: > + Container labels, `` being the label name, the value being the label value. + examples: [ 'container.labels.app=nginx' ] diff --git a/internal/otelschema/_testdata/model/registry/db.yaml b/internal/otelschema/_testdata/model/registry/db.yaml new file mode 100644 index 00000000..3ec2d84c --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/db.yaml @@ -0,0 +1,424 @@ +groups: + - id: registry.db + prefix: db + type: attribute_group + brief: > + This document defines the attributes used to describe telemetry in the context of databases. + attributes: + - id: cassandra.coordinator.dc + type: string + brief: > + The data center of the coordinating node for a query. + examples: 'us-west-2' + tag: tech-specific-cassandra + - id: cassandra.coordinator.id + type: string + brief: > + The ID of the coordinating node for a query. + examples: 'be13faa2-8574-4d71-926d-27f16cf8a7af' + tag: tech-specific-cassandra + - id: cassandra.consistency_level + brief: > + The consistency level of the query. Based on consistency values from [CQL](https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/dml/dmlConfigConsistency.html). + type: + members: + - id: all + value: 'all' + - id: each_quorum + value: 'each_quorum' + - id: quorum + value: 'quorum' + - id: local_quorum + value: 'local_quorum' + - id: one + value: 'one' + - id: two + value: 'two' + - id: three + value: 'three' + - id: local_one + value: 'local_one' + - id: any + value: 'any' + - id: serial + value: 'serial' + - id: local_serial + value: 'local_serial' + tag: tech-specific-cassandra + - id: cassandra.idempotence + type: boolean + brief: > + Whether or not the query is idempotent. + tag: tech-specific-cassandra + - id: cassandra.page_size + type: int + brief: > + The fetch size used for paging, i.e. how many rows will be returned at once. + examples: [5000] + tag: tech-specific-cassandra + - id: cassandra.speculative_execution_count + type: int + brief: > + The number of times a query was speculatively executed. Not set or `0` if the query was not executed speculatively. + examples: [0, 2] + tag: tech-specific-cassandra + - id: cassandra.table + type: string + brief: The name of the primary Cassandra table that the operation is acting upon, including the keyspace name (if applicable). + note: > + This mirrors the db.sql.table attribute but references cassandra rather than sql. + It is not recommended to attempt any client-side parsing of + `db.statement` just to get this property, but it should be set if + it is provided by the library being instrumented. + If the operation is acting upon an anonymous table, or more than one table, this + value MUST NOT be set. + examples: 'mytable' + tag: tech-specific-cassandra + - id: connection_string + type: string + brief: > + The connection string used to connect to the database. + It is recommended to remove embedded credentials. + examples: 'Server=(localdb)\v11.0;Integrated Security=true;' + tag: db-generic + - id: cosmosdb.client_id + type: string + brief: Unique Cosmos client instance id. + examples: '3ba4827d-4422-483f-b59f-85b74211c11d' + tag: tech-specific-cosmosdb + - id: cosmosdb.connection_mode + type: + allow_custom_values: false + members: + - id: gateway + value: 'gateway' + brief: Gateway (HTTP) connections mode + - id: direct + value: 'direct' + brief: Direct connection. + brief: Cosmos client connection mode. + tag: tech-specific-cosmosdb + - id: cosmosdb.container + type: string + brief: Cosmos DB container name. + examples: 'anystring' + tag: tech-specific-cosmosdb + - id: cosmosdb.operation_type + type: + allow_custom_values: true + members: + - id: invalid + value: 'Invalid' + - id: create + value: 'Create' + - id: patch + value: 'Patch' + - id: read + value: 'Read' + - id: read_feed + value: 'ReadFeed' + - id: delete + value: 'Delete' + - id: replace + value: 'Replace' + - id: execute + value: 'Execute' + - id: query + value: 'Query' + - id: head + value: 'Head' + - id: head_feed + value: 'HeadFeed' + - id: upsert + value: 'Upsert' + - id: batch + value: 'Batch' + - id: query_plan + value: 'QueryPlan' + - id: execute_javascript + value: 'ExecuteJavaScript' + brief: CosmosDB Operation Type. + tag: tech-specific-cosmosdb + - id: cosmosdb.request_charge + type: double + brief: RU consumed for that operation + examples: [46.18, 1.0] + tag: tech-specific-cosmosdb + - id: cosmosdb.request_content_length + type: int + brief: Request payload size in bytes + tag: tech-specific-cosmosdb + - id: cosmosdb.status_code + type: int + brief: Cosmos DB status code. + examples: [200, 201] + tag: tech-specific-cosmosdb + - id: cosmosdb.sub_status_code + type: int + brief: Cosmos DB sub status code. + examples: [1000, 1002] + tag: tech-specific-cosmosdb + - id: elasticsearch.cluster.name + type: string + brief: > + Represents the identifier of an Elasticsearch cluster. + examples: ["e9106fc68e3044f0b1475b04bf4ffd5f"] + tag: tech-specific-elasticsearch + - id: elasticsearch.node.name + type: string + brief: > + Represents the human-readable identifier of the node/instance to which a request was routed. + examples: ["instance-0000000001"] + tag: tech-specific-elasticsearch + - id: elasticsearch.path_parts + type: template[string] + brief: > + A dynamic value in the url path. + note: > + Many Elasticsearch url paths allow dynamic values. These SHOULD be recorded in span attributes in the format + `db.elasticsearch.path_parts.`, where `` is the url path part name. The implementation SHOULD + reference the [elasticsearch schema](https://raw.githubusercontent.com/elastic/elasticsearch-specification/main/output/schema/schema.json) + in order to map the path part values to their names. + examples: ['db.elasticsearch.path_parts.index=test-index', 'db.elasticsearch.path_parts.doc_id=123'] + tag: tech-specific-elasticsearch + - id: jdbc.driver_classname + type: string + brief: > + The fully-qualified class name of the [Java Database Connectivity (JDBC)](https://docs.oracle.com/javase/8/docs/technotes/guides/jdbc/) driver used to connect. + examples: ['org.postgresql.Driver', 'com.microsoft.sqlserver.jdbc.SQLServerDriver'] + tag: tech-specific-jdbc + - id: mongodb.collection + type: string + brief: > + The MongoDB collection being accessed within the database stated in `db.name`. + examples: [ 'customers', 'products' ] + tag: tech-specific-mongodb + - id: mssql.instance_name + type: string + note: > + If setting a `db.mssql.instance_name`, `server.port` is no longer + required (but still recommended if non-standard). + brief: > + The Microsoft SQL Server [instance name](https://docs.microsoft.com/sql/connect/jdbc/building-the-connection-url?view=sql-server-ver15) + connecting to. This name is used to determine the port of a named instance. + examples: 'MSSQLSERVER' + tag: tech-specific-mssql + - id: name + type: string + brief: > + This attribute is used to report the name of the database being accessed. + For commands that switch the database, this should be set to the target database + (even if the command fails). + note: > + In some SQL databases, the database name to be used is called "schema name". + In case there are multiple layers that could be considered for database name + (e.g. Oracle instance name and schema name), + the database name to be used is the more specific layer (e.g. Oracle schema name). + examples: [ 'customers', 'main' ] + tag: db-generic + - id: operation + type: string + brief: > + The name of the operation being executed, e.g. the [MongoDB command name](https://docs.mongodb.com/manual/reference/command/#database-operations) + such as `findAndModify`, or the SQL keyword. + note: > + When setting this to an SQL keyword, it is not recommended to + attempt any client-side parsing of `db.statement` just to get this + property, but it should be set if the operation name is provided by + the library being instrumented. + If the SQL statement has an ambiguous operation, or performs more + than one operation, this value may be omitted. + examples: ['findAndModify', 'HMSET', 'SELECT'] + tag: db-generic + - id: redis.database_index + type: int + brief: > + The index of the database being accessed as used in the [`SELECT` command](https://redis.io/commands/select), provided as an integer. + To be used instead of the generic `db.name` attribute. + examples: [0, 1, 15] + tag: tech-specific-redis + - id: sql.table + type: string + brief: The name of the primary table that the operation is acting upon, including the database name (if applicable). + note: > + It is not recommended to attempt any client-side parsing of + `db.statement` just to get this property, but it should be set if + it is provided by the library being instrumented. + If the operation is acting upon an anonymous table, or more than one table, this + value MUST NOT be set. + examples: ['public.users', 'customers'] + tag: tech-specific-sql + - id: statement + type: string + brief: > + The database statement being executed. + examples: ['SELECT * FROM wuser_table', 'SET mykey "WuValue"'] + tag: db-generic + - id: system + brief: An identifier for the database management system (DBMS) product being used. See below for a list of well-known identifiers. + type: + allow_custom_values: true + members: + - id: other_sql + value: 'other_sql' + brief: 'Some other SQL database. Fallback only. See notes.' + - id: mssql + value: 'mssql' + brief: 'Microsoft SQL Server' + - id: mssqlcompact + value: 'mssqlcompact' + brief: 'Microsoft SQL Server Compact' + - id: mysql + value: 'mysql' + brief: 'MySQL' + - id: oracle + value: 'oracle' + brief: 'Oracle Database' + - id: db2 + value: 'db2' + brief: 'IBM Db2' + - id: postgresql + value: 'postgresql' + brief: 'PostgreSQL' + - id: redshift + value: 'redshift' + brief: 'Amazon Redshift' + - id: hive + value: 'hive' + brief: 'Apache Hive' + - id: cloudscape + value: 'cloudscape' + brief: 'Cloudscape' + - id: hsqldb + value: 'hsqldb' + brief: 'HyperSQL DataBase' + - id: progress + value: 'progress' + brief: 'Progress Database' + - id: maxdb + value: 'maxdb' + brief: 'SAP MaxDB' + - id: hanadb + value: 'hanadb' + brief: 'SAP HANA' + - id: ingres + value: 'ingres' + brief: 'Ingres' + - id: firstsql + value: 'firstsql' + brief: 'FirstSQL' + - id: edb + value: 'edb' + brief: 'EnterpriseDB' + - id: cache + value: 'cache' + brief: 'InterSystems Caché' + - id: adabas + value: 'adabas' + brief: 'Adabas (Adaptable Database System)' + - id: firebird + value: 'firebird' + brief: 'Firebird' + - id: derby + value: 'derby' + brief: 'Apache Derby' + - id: filemaker + value: 'filemaker' + brief: 'FileMaker' + - id: informix + value: 'informix' + brief: 'Informix' + - id: instantdb + value: 'instantdb' + brief: 'InstantDB' + - id: interbase + value: 'interbase' + brief: 'InterBase' + - id: mariadb + value: 'mariadb' + brief: 'MariaDB' + - id: netezza + value: 'netezza' + brief: 'Netezza' + - id: pervasive + value: 'pervasive' + brief: 'Pervasive PSQL' + - id: pointbase + value: 'pointbase' + brief: 'PointBase' + - id: sqlite + value: 'sqlite' + brief: 'SQLite' + - id: sybase + value: 'sybase' + brief: 'Sybase' + - id: teradata + value: 'teradata' + brief: 'Teradata' + - id: vertica + value: 'vertica' + brief: 'Vertica' + - id: h2 + value: 'h2' + brief: 'H2' + - id: coldfusion + value: 'coldfusion' + brief: 'ColdFusion IMQ' + - id: cassandra + value: 'cassandra' + brief: 'Apache Cassandra' + - id: hbase + value: 'hbase' + brief: 'Apache HBase' + - id: mongodb + value: 'mongodb' + brief: 'MongoDB' + - id: redis + value: 'redis' + brief: 'Redis' + - id: couchbase + value: 'couchbase' + brief: 'Couchbase' + - id: couchdb + value: 'couchdb' + brief: 'CouchDB' + - id: cosmosdb + value: 'cosmosdb' + brief: 'Microsoft Azure Cosmos DB' + - id: dynamodb + value: 'dynamodb' + brief: 'Amazon DynamoDB' + - id: neo4j + value: 'neo4j' + brief: 'Neo4j' + - id: geode + value: 'geode' + brief: 'Apache Geode' + - id: elasticsearch + value: 'elasticsearch' + brief: 'Elasticsearch' + - id: memcached + value: 'memcached' + brief: 'Memcached' + - id: cockroachdb + value: 'cockroachdb' + brief: 'CockroachDB' + - id: opensearch + value: 'opensearch' + brief: 'OpenSearch' + - id: clickhouse + value: 'clickhouse' + brief: 'ClickHouse' + - id: spanner + value: 'spanner' + brief: 'Cloud Spanner' + - id: trino + value: 'trino' + brief: 'Trino' + tag: db-generic + - id: user + type: string + brief: > + Username for accessing the database. + examples: ['readonly_user', 'reporting_user'] + tag: db-generic diff --git a/internal/otelschema/_testdata/model/registry/deprecated/http.yaml b/internal/otelschema/_testdata/model/registry/deprecated/http.yaml new file mode 100644 index 00000000..91503286 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/deprecated/http.yaml @@ -0,0 +1,71 @@ +groups: + - id: attributes.http.deprecated + type: attribute_group + brief: "Describes deprecated HTTP attributes." + prefix: http + attributes: + - id: method + type: string + brief: 'Deprecated, use `http.request.method` instead.' + stability: deprecated + examples: ["GET", "POST", "HEAD"] + - id: status_code + type: int + brief: 'Deprecated, use `http.response.status_code` instead.' + stability: deprecated + examples: [200] + - id: scheme + type: string + brief: 'Deprecated, use `url.scheme` instead.' + stability: deprecated + examples: ['http', 'https'] + - id: url + type: string + brief: 'Deprecated, use `url.full` instead.' + stability: deprecated + examples: ['https://www.foo.bar/search?q=OpenTelemetry#SemConv'] + - id: target + type: string + brief: 'Deprecated, use `url.path` and `url.query` instead.' + stability: deprecated + examples: ['/search?q=OpenTelemetry#SemConv'] + - id: request_content_length + type: int + brief: 'Deprecated, use `http.request.header.content-length` instead.' + stability: deprecated + examples: 3495 + - id: response_content_length + type: int + brief: 'Deprecated, use `http.response.header.content-length` instead.' + stability: deprecated + examples: 3495 + - id: flavor + type: + allow_custom_values: true + members: + - id: http_1_0 + value: '1.0' + brief: 'HTTP/1.0' + - id: http_1_1 + value: '1.1' + brief: 'HTTP/1.1' + - id: http_2_0 + value: '2.0' + brief: 'HTTP/2' + - id: http_3_0 + value: '3.0' + brief: 'HTTP/3' + - id: spdy + value: 'SPDY' + brief: 'SPDY protocol.' + - id: quic + value: 'QUIC' + brief: 'QUIC protocol.' + brief: 'Deprecated, use `network.protocol.name` instead.' + stability: deprecated + - id: user_agent + type: string + brief: 'Deprecated, use `user_agent.original` instead.' + examples: ['CERN-LineMode/2.15 libwww/2.17b3', + 'Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1'] + stability: deprecated diff --git a/internal/otelschema/_testdata/model/registry/deprecated/network.yaml b/internal/otelschema/_testdata/model/registry/deprecated/network.yaml new file mode 100644 index 00000000..19e9c226 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/deprecated/network.yaml @@ -0,0 +1,100 @@ +groups: + - id: network-deprecated + prefix: net + type: attribute_group + brief: > + These attributes may be used for any network related operation. + attributes: + - id: sock.peer.name + type: string + stability: deprecated + brief: Deprecated, no replacement at this time. + examples: ['/var/my.sock'] + - id: sock.peer.addr + type: string + stability: deprecated + brief: Deprecated, use `network.peer.address`. + examples: ['192.168.0.1'] + - id: sock.peer.port + type: int + stability: deprecated + examples: [65531] + brief: Deprecated, use `network.peer.port`. + - id: peer.name + type: string + stability: deprecated + brief: Deprecated, use `server.address` on client spans and `client.address` on server spans. + examples: ['example.com'] + - id: peer.port + type: int + stability: deprecated + brief: Deprecated, use `server.port` on client spans and `client.port` on server spans. + examples: [8080] + - id: host.name + type: string + stability: deprecated + brief: Deprecated, use `server.address`. + examples: ['example.com'] + - id: host.port + type: int + stability: deprecated + brief: Deprecated, use `server.port`. + examples: [8080] + - id: sock.host.addr + type: string + stability: deprecated + brief: Deprecated, use `network.local.address`. + examples: ['/var/my.sock'] + - id: sock.host.port + type: int + stability: deprecated + brief: Deprecated, use `network.local.port`. + examples: [8080] + - id: transport + type: + allow_custom_values: true + members: + - id: ip_tcp + value: "ip_tcp" + - id: ip_udp + value: "ip_udp" + - id: pipe + value: "pipe" + brief: 'Named or anonymous pipe.' + - id: inproc + value: "inproc" + brief: 'In-process communication.' + note: > + Signals that there is only in-process communication not using a "real" network protocol + in cases where network attributes would normally be expected. Usually all other network + attributes can be left out in that case. + - id: other + value: "other" + brief: 'Something else (non IP-based).' + stability: deprecated + brief: Deprecated, use `network.transport`. + - id: protocol.name + type: string + stability: deprecated + brief: Deprecated, use `network.protocol.name`. + examples: ['amqp', 'http', 'mqtt'] + - id: protocol.version + type: string + stability: deprecated + brief: Deprecated, use `network.protocol.version`. + examples: '3.1.1' + - id: sock.family + type: + allow_custom_values: true + members: + - id: inet + value: 'inet' + brief: "IPv4 address" + - id: inet6 + value: 'inet6' + brief: "IPv6 address" + - id: unix + value: 'unix' + brief: "Unix domain socket path" + stability: deprecated + brief: Deprecated, use `network.transport` and `network.type`. diff --git a/internal/otelschema/_testdata/model/registry/destination.yaml b/internal/otelschema/_testdata/model/registry/destination.yaml new file mode 100644 index 00000000..595a1c67 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/destination.yaml @@ -0,0 +1,23 @@ +groups: + - id: destination + prefix: destination + type: attribute_group + brief: > + These attributes may be used to describe the receiver of a network exchange/packet. These should be used + when there is no client/server relationship between the two sides, or when that relationship is unknown. + This covers low-level network interactions (e.g. packet tracing) where you don't know if + there was a connection or which side initiated it. + This also covers unidirectional UDP flows and peer-to-peer communication where the + "user-facing" surface of the protocol / API doesn't expose a clear notion of client and server. + attributes: + - id: address + type: string + brief: "Destination address - domain name if available without reverse DNS lookup; otherwise, IP address or Unix domain socket name." + note: > + When observed from the source side, and when communicating through an intermediary, `destination.address` SHOULD represent + the destination address behind any intermediaries, for example proxies, if it's available. + examples: ['destination.example.com', '10.1.2.80', '/tmp/my.sock'] + - id: port + type: int + brief: 'Destination port number' + examples: [3389, 2888] diff --git a/internal/otelschema/_testdata/model/registry/device.yaml b/internal/otelschema/_testdata/model/registry/device.yaml new file mode 100644 index 00000000..6eaf8e3b --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/device.yaml @@ -0,0 +1,46 @@ +groups: + - id: registry.device + prefix: device + type: resource + brief: > + Describes device attributes. + attributes: + - id: id + type: string + brief: > + A unique identifier representing the device + note: > + The device identifier MUST only be defined using the values outlined below. This value is not an advertising + identifier and MUST NOT be used as such. + On iOS (Swift or Objective-C), this value MUST be equal to the [vendor identifier](https://developer.apple.com/documentation/uikit/uidevice/1620059-identifierforvendor). + On Android (Java or Kotlin), this value MUST be equal to the Firebase Installation ID or a globally unique + UUID which is persisted across sessions in your application. More information can be found [here](https://developer.android.com/training/articles/user-data-ids) + on best practices and exact implementation details. + Caution should be taken when storing personal data or anything which can identify a user. GDPR and + data protection laws may apply, ensure you do your own due diligence. + examples: ['2ab2916d-a51f-4ac8-80ee-45ac31a28092'] + - id: manufacturer + type: string + brief: > + The name of the device manufacturer + note: > + The Android OS provides this field via [Build](https://developer.android.com/reference/android/os/Build#MANUFACTURER). + iOS apps SHOULD hardcode the value `Apple`. + examples: ['Apple', 'Samsung'] + - id: model.identifier + type: string + brief: > + The model identifier for the device + note: > + It's recommended this value represents a machine-readable version of + the model identifier rather than the market or consumer-friendly name + of the device. + examples: ['iPhone3,4', 'SM-G920F'] + - id: model.name + type: string + brief: > + The marketing name for the device model + note: > + It's recommended this value represents a human-readable version of the + device model rather than a machine-readable alternative. + examples: ['iPhone 6s Plus', 'Samsung Galaxy S6'] diff --git a/internal/otelschema/_testdata/model/registry/disk.yaml b/internal/otelschema/_testdata/model/registry/disk.yaml new file mode 100644 index 00000000..90d6fb27 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/disk.yaml @@ -0,0 +1,17 @@ +groups: + - id: registry.disk + prefix: disk + type: attribute_group + brief: > + These attributes may be used for any disk related operation. + attributes: + - id: io.direction + type: + allow_custom_values: false + members: + - id: read + value: 'read' + - id: write + value: 'write' + brief: "The disk IO operation direction." + examples: ["read"] diff --git a/internal/otelschema/_testdata/model/registry/error.yaml b/internal/otelschema/_testdata/model/registry/error.yaml new file mode 100644 index 00000000..683012e3 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/error.yaml @@ -0,0 +1,35 @@ +groups: + - id: registry.error + type: attribute_group + prefix: error + brief: > + This document defines the shared attributes used to report an error. + attributes: + - id: type + stability: stable + brief: > + Describes a class of error the operation ended with. + type: + allow_custom_values: true + members: + - id: other + value: "_OTHER" + brief: > + A fallback error value to be used when the instrumentation doesn't define a custom value. + examples: ['timeout', 'java.net.UnknownHostException', 'server_certificate_invalid', '500'] + note: | + The `error.type` SHOULD be predictable and SHOULD have low cardinality. + Instrumentations SHOULD document the list of errors they report. + + The cardinality of `error.type` within one instrumentation library SHOULD be low. + Telemetry consumers that aggregate data from multiple instrumentation libraries and applications + should be prepared for `error.type` to have high cardinality at query time when no + additional filters are applied. + + If the operation has completed successfully, instrumentations SHOULD NOT set `error.type`. + + If a specific domain defines its own set of error identifiers (such as HTTP or gRPC status codes), + it's RECOMMENDED to: + + * Use a domain-specific attribute + * Set `error.type` to capture all errors, regardless of whether they are defined within the domain-specific set or not. diff --git a/internal/otelschema/_testdata/model/registry/host.yaml b/internal/otelschema/_testdata/model/registry/host.yaml new file mode 100644 index 00000000..3deb5f57 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/host.yaml @@ -0,0 +1,122 @@ +groups: + - id: registry.host + prefix: host + type: resource + brief: > + A host is defined as a computing instance. For example, physical servers, virtual machines, switches or disk array. + attributes: + - id: id + type: string + brief: > + Unique host ID. For Cloud, this must be the instance_id assigned by the cloud provider. + For non-containerized systems, this should be the `machine-id`. See the table below for + the sources to use to determine the `machine-id` based on operating system. + examples: ['fdbf79e8af94cb7f9e8df36789187052'] + - id: name + type: string + brief: > + Name of the host. On Unix systems, it may contain what the hostname + command returns, or the fully qualified hostname, or another name + specified by the user. + examples: ['opentelemetry-test'] + - id: type + type: string + brief: > + Type of host. For Cloud, this must be the machine type. + examples: ['n1-standard-1'] + - id: arch + type: + allow_custom_values: true + members: + - id: amd64 + value: 'amd64' + brief: "AMD64" + - id: arm32 + value: 'arm32' + brief: "ARM32" + - id: arm64 + value: 'arm64' + brief: "ARM64" + - id: ia64 + value: 'ia64' + brief: "Itanium" + - id: ppc32 + value: 'ppc32' + brief: "32-bit PowerPC" + - id: ppc64 + value: 'ppc64' + brief: "64-bit PowerPC" + - id: s390x + value: 's390x' + brief: "IBM z/Architecture" + - id: x86 + value: 'x86' + brief: "32-bit x86" + brief: > + The CPU architecture the host system is running on. + - id: image.name + type: string + brief: > + Name of the VM image or OS install the host was instantiated from. + examples: ['infra-ami-eks-worker-node-7d4ec78312', 'CentOS-8-x86_64-1905'] + - id: image.id + type: string + brief: > + VM image ID or host OS image ID. For Cloud, this value is from the provider. + examples: ['ami-07b06b442921831e5'] + - id: image.version + type: string + brief: > + The version string of the VM image or host OS as defined in + [Version Attributes](/docs/resource/README.md#version-attributes). + examples: ['0.1'] + - id: ip + type: string[] + brief: > + Available IP addresses of the host, excluding loopback interfaces. + note: > + IPv4 Addresses MUST be specified in dotted-quad notation. IPv6 addresses + MUST be specified in the [RFC 5952](https://www.rfc-editor.org/rfc/rfc5952.html) format. + examples: ["192.168.1.140", "fe80::abc2:4a28:737a:609e"] + - id: mac + type: string[] + brief: > + Available MAC addresses of the host, excluding loopback interfaces. + note: > + MAC Addresses MUST be represented in [IEEE RA hexadecimal form](https://standards.ieee.org/wp-content/uploads/import/documents/tutorials/eui.pdf): + as hyphen-separated octets in uppercase hexadecimal form from most to least significant. + examples: ['AC-DE-48-23-45-67', 'AC-DE-48-23-45-67-01-9F'] + - id: cpu.vendor.id + type: string + brief: > + Processor manufacturer identifier. A maximum 12-character string. + note: > + [CPUID](https://wiki.osdev.org/CPUID) command returns the vendor ID string in EBX, EDX and ECX registers. + Writing these to memory in this order results in a 12-character string. + examples: [ 'GenuineIntel' ] + - id: cpu.family + type: string + brief: > + Family or generation of the CPU. + examples: [ '6', 'PA-RISC 1.1e' ] + - id: cpu.model.id + type: string + brief: > + Model identifier. It provides more granular information about the CPU, distinguishing it from + other CPUs within the same family. + examples: [ '6', '9000/778/B180L' ] + - id: cpu.model.name + type: string + brief: > + Model designation of the processor. + examples: [ '11th Gen Intel(R) Core(TM) i7-1185G7 @ 3.00GHz' ] + - id: cpu.stepping + type: int + brief: > + Stepping or core revisions. + examples: [ 1 ] + - id: cpu.cache.l2.size + type: int + brief: > + The amount of level 2 memory cache available to the processor (in Bytes). + examples: [ 12288000 ] diff --git a/internal/otelschema/_testdata/model/registry/http.yaml b/internal/otelschema/_testdata/model/registry/http.yaml new file mode 100644 index 00000000..2cb8e8b8 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/http.yaml @@ -0,0 +1,135 @@ +groups: + - id: registry.http + prefix: http + type: attribute_group + brief: 'This document defines semantic convention attributes in the HTTP namespace.' + attributes: + - id: request.body.size + type: int + brief: > + The size of the request payload body in bytes. This is the number of bytes transferred excluding headers and + is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) + header. For requests using transport encoding, this should be the compressed size. + examples: 3495 + stability: experimental # this should not be marked stable with other HTTP attributes + - id: request.header + stability: stable + type: template[string[]] + brief: > + HTTP request headers, `` being the normalized HTTP Header name (lowercase), the value being the header values. + note: > + Instrumentations SHOULD require an explicit configuration of which headers are to be captured. + Including all request headers can be a security risk - explicit configuration helps avoid leaking sensitive information. + + The `User-Agent` header is already captured in the `user_agent.original` attribute. + Users MAY explicitly configure instrumentations to capture them even though it is not recommended. + + The attribute value MUST consist of either multiple header values as an array of strings + or a single-item array containing a possibly comma-concatenated string, depending on the way + the HTTP library provides access to headers. + examples: ['http.request.header.content-type=["application/json"]', 'http.request.header.x-forwarded-for=["1.2.3.4", "1.2.3.5"]'] + - id: request.method + stability: stable + type: + allow_custom_values: true + members: + - id: connect + value: "CONNECT" + brief: 'CONNECT method.' + - id: delete + value: "DELETE" + brief: 'DELETE method.' + - id: get + value: "GET" + brief: 'GET method.' + - id: head + value: "HEAD" + brief: 'HEAD method.' + - id: options + value: "OPTIONS" + brief: 'OPTIONS method.' + - id: patch + value: "PATCH" + brief: 'PATCH method.' + - id: post + value: "POST" + brief: 'POST method.' + - id: put + value: "PUT" + brief: 'PUT method.' + - id: trace + value: "TRACE" + brief: 'TRACE method.' + - id: other + value: "_OTHER" + brief: 'Any HTTP method that the instrumentation has no prior knowledge of.' + brief: 'HTTP request method.' + examples: ["GET", "POST", "HEAD"] + note: | + HTTP request method value SHOULD be "known" to the instrumentation. + By default, this convention defines "known" methods as the ones listed in [RFC9110](https://www.rfc-editor.org/rfc/rfc9110.html#name-methods) + and the PATCH method defined in [RFC5789](https://www.rfc-editor.org/rfc/rfc5789.html). + + If the HTTP request method is not known to instrumentation, it MUST set the `http.request.method` attribute to `_OTHER`. + + If the HTTP instrumentation could end up converting valid HTTP request methods to `_OTHER`, then it MUST provide a way to override + the list of known HTTP methods. If this override is done via environment variable, then the environment variable MUST be named + OTEL_INSTRUMENTATION_HTTP_KNOWN_METHODS and support a comma-separated list of case-sensitive known HTTP methods + (this list MUST be a full override of the default known method, it is not a list of known methods in addition to the defaults). + + HTTP method names are case-sensitive and `http.request.method` attribute value MUST match a known HTTP method name exactly. + Instrumentations for specific web frameworks that consider HTTP methods to be case insensitive, SHOULD populate a canonical equivalent. + Tracing instrumentations that do so, MUST also set `http.request.method_original` to the original value. + - id: request.method_original + stability: stable + type: string + brief: Original HTTP method sent by the client in the request line. + examples: ["GeT", "ACL", "foo"] + - id: request.resend_count + stability: stable + type: int + brief: > + The ordinal number of request resending attempt (for any reason, including redirects). + note: > + The resend count SHOULD be updated each time an HTTP request gets resent by the client, regardless of what + was the cause of the resending (e.g. redirection, authorization failure, 503 Server Unavailable, network issues, + or any other). + examples: 3 + - id: response.body.size + type: int + brief: > + The size of the response payload body in bytes. This is the number of bytes transferred excluding headers and + is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) + header. For requests using transport encoding, this should be the compressed size. + examples: 3495 + stability: experimental # this should not be marked stable with other HTTP attributes + - id: response.header + stability: stable + type: template[string[]] + brief: > + HTTP response headers, `` being the normalized HTTP Header name (lowercase), the value being the header values. + note: > + Instrumentations SHOULD require an explicit configuration of which headers are to be captured. + Including all response headers can be a security risk - explicit configuration helps avoid leaking sensitive information. + + Users MAY explicitly configure instrumentations to capture them even though it is not recommended. + + The attribute value MUST consist of either multiple header values as an array of strings + or a single-item array containing a possibly comma-concatenated string, depending on the way + the HTTP library provides access to headers. + examples: ['http.response.header.content-type=["application/json"]', 'http.response.header.my-custom-header=["abc", "def"]'] + - id: response.status_code + stability: stable + type: int + brief: '[HTTP response status code](https://tools.ietf.org/html/rfc7231#section-6).' + examples: [200] + - id: route + stability: stable + type: string + brief: > + The matched route, that is, the path template in the format used by the respective server framework. + examples: ['/users/:userID?', '{controller}/{action}/{id?}'] + note: > + MUST NOT be populated when this is not supported by the HTTP server framework as the route attribute should have low-cardinality and the URI path can NOT substitute it. + + SHOULD include the [application root](/docs/http/http-spans.md#http-server-definitions) if there is one. diff --git a/internal/otelschema/_testdata/model/registry/k8s.yaml b/internal/otelschema/_testdata/model/registry/k8s.yaml new file mode 100644 index 00000000..c42744cb --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/k8s.yaml @@ -0,0 +1,140 @@ +groups: + - id: registry.k8s + prefix: k8s + type: resource + brief: > + Kubernetes resource attributes. + attributes: + - id: cluster.name + type: string + brief: > + The name of the cluster. + examples: ['opentelemetry-cluster'] + - id: cluster.uid + type: string + brief: > + A pseudo-ID for the cluster, set to the UID of the `kube-system` + namespace. + note: | + K8s doesn't have support for obtaining a cluster ID. If this is ever + added, we will recommend collecting the `k8s.cluster.uid` through the + official APIs. In the meantime, we are able to use the `uid` of the + `kube-system` namespace as a proxy for cluster ID. Read on for the + rationale. + + Every object created in a K8s cluster is assigned a distinct UID. The + `kube-system` namespace is used by Kubernetes itself and will exist + for the lifetime of the cluster. Using the `uid` of the `kube-system` + namespace is a reasonable proxy for the K8s ClusterID as it will only + change if the cluster is rebuilt. Furthermore, Kubernetes UIDs are + UUIDs as standardized by + [ISO/IEC 9834-8 and ITU-T X.667](https://www.itu.int/ITU-T/studygroups/com17/oid.html). + Which states: + + > If generated according to one of the mechanisms defined in Rec. + ITU-T X.667 | ISO/IEC 9834-8, a UUID is either guaranteed to be + different from all other UUIDs generated before 3603 A.D., or is + extremely likely to be different (depending on the mechanism chosen). + + Therefore, UIDs between clusters should be extremely unlikely to + conflict. + examples: ['218fc5a9-a5f1-4b54-aa05-46717d0ab26d'] + - id: node.name + type: string + brief: > + The name of the Node. + examples: ['node-1'] + - id: node.uid + type: string + brief: > + The UID of the Node. + examples: ['1eb3a0c6-0477-4080-a9cb-0cb7db65c6a2'] + - id: namespace.name + type: string + brief: > + The name of the namespace that the pod is running in. + examples: ['default'] + - id: pod.uid + type: string + brief: > + The UID of the Pod. + examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] + - id: pod.name + type: string + brief: > + The name of the Pod. + examples: ['opentelemetry-pod-autoconf'] + - id: container.name + type: string + brief: > + The name of the Container from Pod specification, must be unique + within a Pod. Container runtime usually uses different globally unique + name (`container.name`). + examples: ['redis'] + - id: container.restart_count + type: int + brief: > + Number of times the container was restarted. This attribute can be + used to identify a particular container (running or stopped) within a + container spec. + examples: [0, 2] + - id: replicaset.uid + type: string + brief: > + The UID of the ReplicaSet. + examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] + - id: replicaset.name + type: string + brief: > + The name of the ReplicaSet. + examples: ['opentelemetry'] + - id: deployment.uid + type: string + brief: > + The UID of the Deployment. + examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] + - id: deployment.name + type: string + brief: > + The name of the Deployment. + examples: ['opentelemetry'] + - id: statefulset.uid + type: string + brief: > + The UID of the StatefulSet. + examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] + - id: statefulset.name + type: string + brief: > + The name of the StatefulSet. + examples: ['opentelemetry'] + - id: daemonset.uid + type: string + brief: > + The UID of the DaemonSet. + examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] + - id: daemonset.name + type: string + brief: > + The name of the DaemonSet. + examples: ['opentelemetry'] + - id: job.uid + type: string + brief: > + The UID of the Job. + examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] + - id: job.name + type: string + brief: > + The name of the Job. + examples: ['opentelemetry'] + - id: cronjob.uid + type: string + brief: > + The UID of the CronJob. + examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] + - id: cronjob.name + type: string + brief: > + The name of the CronJob. + examples: ['opentelemetry'] diff --git a/internal/otelschema/_testdata/model/registry/messaging.yaml b/internal/otelschema/_testdata/model/registry/messaging.yaml new file mode 100644 index 00000000..c7ba8fd4 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/messaging.yaml @@ -0,0 +1,245 @@ +groups: + - id: registry.messaging + prefix: messaging + type: attribute_group + brief: 'Attributes describing telemetry around messaging systems and messaging activities.' + attributes: + - id: batch.message_count + type: int + brief: The number of messages sent, received, or processed in the scope of the batching operation. + note: > + Instrumentations SHOULD NOT set `messaging.batch.message_count` on spans that operate with a single message. + When a messaging client library supports both batch and single-message API for the same operation, instrumentations SHOULD + use `messaging.batch.message_count` for batching APIs and SHOULD NOT use it for single-message APIs. + examples: [0, 1, 2] + - id: client_id + type: string + brief: > + A unique identifier for the client that consumes or produces a message. + examples: ['client-5', 'myhost@8742@s8083jm'] + - id: destination.name + type: string + brief: 'The message destination name' + note: | + Destination name SHOULD uniquely identify a specific queue, topic or other entity within the broker. If + the broker doesn't have such notion, the destination name SHOULD uniquely identify the broker. + examples: ['MyQueue', 'MyTopic'] + - id: destination.template + type: string + brief: Low cardinality representation of the messaging destination name + note: > + Destination names could be constructed from templates. + An example would be a destination name involving a user name or product id. + Although the destination name in this case is of high cardinality, + the underlying template is of low cardinality and can be effectively + used for grouping and aggregation. + examples: ['/customers/{customerId}'] + - id: destination.anonymous + type: boolean + brief: 'A boolean that is true if the message destination is anonymous (could be unnamed or have auto-generated name).' + - id: destination.temporary + type: boolean + brief: 'A boolean that is true if the message destination is temporary and might not exist anymore after messages are processed.' + - id: destination_publish.anonymous + type: boolean + brief: 'A boolean that is true if the publish message destination is anonymous (could be unnamed or have auto-generated name).' + - id: destination_publish.name + type: string + brief: 'The name of the original destination the message was published to' + note: | + The name SHOULD uniquely identify a specific queue, topic, or other entity within the broker. If + the broker doesn't have such notion, the original destination name SHOULD uniquely identify the broker. + examples: ['MyQueue', 'MyTopic'] + - id: kafka.consumer.group + type: string + brief: > + Name of the Kafka Consumer Group that is handling the message. + Only applies to consumers, not producers. + examples: 'my-group' + - id: kafka.destination.partition + type: int + brief: > + Partition the message is sent to. + examples: 2 + - id: kafka.message.key + type: string + brief: > + Message keys in Kafka are used for grouping alike messages to ensure they're processed on the same partition. + They differ from `messaging.message.id` in that they're not unique. + If the key is `null`, the attribute MUST NOT be set. + note: > + If the key type is not string, it's string representation has to be supplied for the attribute. + If the key has no unambiguous, canonical string form, don't include its value. + examples: 'myKey' + - id: kafka.message.offset + type: int + brief: > + The offset of a record in the corresponding Kafka partition. + examples: 42 + - id: kafka.message.tombstone + type: boolean + brief: 'A boolean that is true if the message is a tombstone.' + - id: message.conversation_id + type: string + brief: > + The conversation ID identifying the conversation to which the message belongs, + represented as a string. Sometimes called "Correlation ID". + examples: 'MyConversationId' + - id: message.envelope.size + type: int + brief: > + The size of the message body and metadata in bytes. + note: | + This can refer to both the compressed or uncompressed size. If both sizes are known, the uncompressed + size should be used. + examples: 2738 + - id: message.id + type: string + brief: 'A value used by the messaging system as an identifier for the message, represented as a string.' + examples: '452a7c7c7c7048c2f887f61572b18fc2' + - id: message.body.size + type: int + brief: > + The size of the message body in bytes. + note: | + This can refer to both the compressed or uncompressed body size. If both sizes are known, the uncompressed + body size should be used. + examples: 1439 + - id: operation + type: + allow_custom_values: true + members: + - id: publish + value: "publish" + brief: > + One or more messages are provided for publishing to an intermediary. + If a single message is published, the context of the "Publish" span can be used as the creation context and no "Create" span needs to be created. + - id: create + value: "create" + brief: > + A message is created. + "Create" spans always refer to a single message and are used to provide a unique creation context for messages in batch publishing scenarios. + - id: receive + value: "receive" + brief: > + One or more messages are requested by a consumer. + This operation refers to pull-based scenarios, where consumers explicitly call methods of messaging SDKs to receive messages. + - id: deliver + value: "deliver" + brief: > + One or more messages are passed to a consumer. + This operation refers to push-based scenarios, where consumer register callbacks which get called by messaging SDKs. + brief: > + A string identifying the kind of messaging operation. + note: If a custom value is used, it MUST be of low cardinality. + - id: rabbitmq.destination.routing_key + type: string + brief: > + RabbitMQ message routing key. + examples: 'myKey' + - id: rocketmq.client_group + type: string + brief: > + Name of the RocketMQ producer/consumer group that is handling the message. The client type is identified by the SpanKind. + examples: 'myConsumerGroup' + - id: rocketmq.consumption_model + type: + allow_custom_values: false + members: + - id: clustering + value: 'clustering' + brief: 'Clustering consumption model' + - id: broadcasting + value: 'broadcasting' + brief: 'Broadcasting consumption model' + brief: > + Model of message consumption. This only applies to consumer spans. + - id: rocketmq.message.delay_time_level + type: int + brief: > + The delay time level for delay message, which determines the message delay time. + examples: 3 + - id: rocketmq.message.delivery_timestamp + type: int + brief: > + The timestamp in milliseconds that the delay message is expected to be delivered to consumer. + examples: 1665987217045 + - id: rocketmq.message.group + type: string + brief: > + It is essential for FIFO message. Messages that belong to the same message group are always processed one by one within the same consumer group. + examples: 'myMessageGroup' + - id: rocketmq.message.keys + type: string[] + brief: > + Key(s) of message, another way to mark message besides message id. + examples: ['keyA', 'keyB'] + - id: rocketmq.message.tag + type: string + brief: > + The secondary classifier of message besides topic. + examples: tagA + - id: rocketmq.message.type + type: + allow_custom_values: false + members: + - id: normal + value: 'normal' + brief: "Normal message" + - id: fifo + value: 'fifo' + brief: 'FIFO message' + - id: delay + value: 'delay' + brief: 'Delay message' + - id: transaction + value: 'transaction' + brief: 'Transaction message' + brief: > + Type of message. + - id: rocketmq.namespace + type: string + brief: > + Namespace of RocketMQ resources, resources in different namespaces are individual. + examples: 'myNamespace' + - id: gcp_pubsub.message.ordering_key + type: string + brief: > + The ordering key for a given message. If the attribute is not present, the message does not have an ordering key. + examples: 'ordering_key' + - id: system + brief: > + An identifier for the messaging system being used. See below for a list of well-known identifiers. + type: + allow_custom_values: true + members: + - id: activemq + value: 'activemq' + brief: 'Apache ActiveMQ' + - id: aws_sqs + value: 'aws_sqs' + brief: 'Amazon Simple Queue Service (SQS)' + - id: azure_eventgrid + value: 'azure_eventgrid' + brief: 'Azure Event Grid' + - id: azure_eventhubs + value: 'azure_eventhubs' + brief: 'Azure Event Hubs' + - id: azure_servicebus + value: 'azure_servicebus' + brief: 'Azure Service Bus' + - id: gcp_pubsub + value: 'gcp_pubsub' + brief: 'Google Cloud Pub/Sub' + - id: jms + value: 'jms' + brief: 'Java Message Service' + - id: kafka + value: 'kafka' + brief: 'Apache Kafka' + - id: rabbitmq + value: 'rabbitmq' + brief: 'RabbitMQ' + - id: rocketmq + value: 'rocketmq' + brief: 'Apache RocketMQ' diff --git a/internal/otelschema/_testdata/model/registry/network.yaml b/internal/otelschema/_testdata/model/registry/network.yaml new file mode 100644 index 00000000..7715506b --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/network.yaml @@ -0,0 +1,194 @@ +groups: + - id: registry.network + prefix: network + type: attribute_group + brief: > + These attributes may be used for any network related operation. + attributes: + - id: carrier.icc + type: string + brief: "The ISO 3166-1 alpha-2 2-character country code associated with the mobile carrier network." + examples: "DE" + - id: carrier.mcc + type: string + brief: "The mobile carrier country code." + examples: "310" + - id: carrier.mnc + type: string + brief: "The mobile carrier network code." + examples: "001" + - id: carrier.name + type: string + brief: "The name of the mobile carrier." + examples: "sprint" + - id: connection.subtype + type: + allow_custom_values: true + members: + - id: gprs + brief: GPRS + value: "gprs" + - id: edge + brief: EDGE + value: "edge" + - id: umts + brief: UMTS + value: "umts" + - id: cdma + brief: CDMA + value: "cdma" + - id: evdo_0 + brief: EVDO Rel. 0 + value: "evdo_0" + - id: evdo_a + brief: "EVDO Rev. A" + value: "evdo_a" + - id: cdma2000_1xrtt + brief: CDMA2000 1XRTT + value: "cdma2000_1xrtt" + - id: hsdpa + brief: HSDPA + value: "hsdpa" + - id: hsupa + brief: HSUPA + value: "hsupa" + - id: hspa + brief: HSPA + value: "hspa" + - id: iden + brief: IDEN + value: "iden" + - id: evdo_b + brief: "EVDO Rev. B" + value: "evdo_b" + - id: lte + brief: LTE + value: "lte" + - id: ehrpd + brief: EHRPD + value: "ehrpd" + - id: hspap + brief: HSPAP + value: "hspap" + - id: gsm + brief: GSM + value: "gsm" + - id: td_scdma + brief: TD-SCDMA + value: "td_scdma" + - id: iwlan + brief: IWLAN + value: "iwlan" + - id: nr + brief: "5G NR (New Radio)" + value: "nr" + - id: nrnsa + brief: "5G NRNSA (New Radio Non-Standalone)" + value: "nrnsa" + - id: lte_ca + brief: LTE CA + value: "lte_ca" + brief: 'This describes more details regarding the connection.type. It may be the type of cell technology connection, but it could be used for describing details about a wifi connection.' + examples: 'LTE' + - id: connection.type + type: + allow_custom_values: true + members: + - id: wifi + value: "wifi" + - id: wired + value: "wired" + - id: cell + value: "cell" + - id: unavailable + value: "unavailable" + - id: unknown + value: "unknown" + brief: 'The internet connection type.' + examples: 'wifi' + - id: local.address + stability: stable + type: string + brief: Local address of the network connection - IP address or Unix domain socket name. + examples: ['10.1.2.80', '/tmp/my.sock'] + - id: local.port + stability: stable + type: int + brief: Local port number of the network connection. + examples: [65123] + - id: peer.address + stability: stable + type: string + brief: Peer address of the network connection - IP address or Unix domain socket name. + examples: ['10.1.2.80', '/tmp/my.sock'] + - id: peer.port + stability: stable + type: int + brief: Peer port number of the network connection. + examples: [65123] + - id: protocol.name + stability: stable + type: string + brief: '[OSI application layer](https://osi-model.com/application-layer/) or non-OSI equivalent.' + note: The value SHOULD be normalized to lowercase. + examples: ['amqp', 'http', 'mqtt'] + - id: protocol.version + stability: stable + type: string + brief: Version of the protocol specified in `network.protocol.name`. + examples: '3.1.1' + note: > + `network.protocol.version` refers to the version of the protocol used and might be + different from the protocol client's version. If the HTTP client has a version + of `0.27.2`, but sends HTTP version `1.1`, this attribute should be set to `1.1`. + - id: transport + stability: stable + type: + allow_custom_values: true + members: + - id: tcp + value: 'tcp' + brief: "TCP" + - id: udp + value: 'udp' + brief: "UDP" + - id: pipe + value: "pipe" + brief: 'Named or anonymous pipe.' + - id: unix + value: 'unix' + brief: "Unix domain socket" + brief: > + [OSI transport layer](https://osi-model.com/transport-layer/) or + [inter-process communication method](https://wikipedia.org/wiki/Inter-process_communication). + note: | + The value SHOULD be normalized to lowercase. + + Consider always setting the transport when setting a port number, since + a port number is ambiguous without knowing the transport. For example + different processes could be listening on TCP port 12345 and UDP port 12345. + examples: ['tcp', 'udp'] + - id: type + stability: stable + type: + allow_custom_values: true + members: + - id: ipv4 + value: 'ipv4' + brief: "IPv4" + - id: ipv6 + value: 'ipv6' + brief: "IPv6" + brief: '[OSI network layer](https://osi-model.com/network-layer/) or non-OSI equivalent.' + note: The value SHOULD be normalized to lowercase. + examples: ['ipv4', 'ipv6'] + - id: io.direction + type: + allow_custom_values: false + members: + - id: transmit + value: 'transmit' + - id: receive + value: 'receive' + brief: "The network IO operation direction." + examples: ["transmit"] diff --git a/internal/otelschema/_testdata/model/registry/oci.yaml b/internal/otelschema/_testdata/model/registry/oci.yaml new file mode 100644 index 00000000..45e28387 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/oci.yaml @@ -0,0 +1,21 @@ +groups: + - id: registry.oci.manifest + prefix: oci.manifest + type: resource + brief: > + An OCI image manifest. + attributes: + - id: digest + type: string + brief: > + The digest of the OCI image manifest. For container images specifically is the + digest by which the container image is known. + note: > + Follows + [OCI Image Manifest Specification](https://github.com/opencontainers/image-spec/blob/main/manifest.md), + and specifically the + [Digest property](https://github.com/opencontainers/image-spec/blob/main/descriptor.md#digests). + + An example can be found in + [Example Image Manifest](https://docs.docker.com/registry/spec/manifest-v2-2/#example-image-manifest). + examples: [ 'sha256:e4ca62c0d62f3e886e684806dfe9d4e0cda60d54986898173c1083856cfda0f4' ] diff --git a/internal/otelschema/_testdata/model/registry/os.yaml b/internal/otelschema/_testdata/model/registry/os.yaml new file mode 100644 index 00000000..5b699b1f --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/os.yaml @@ -0,0 +1,69 @@ +groups: + - id: registry.os + prefix: os + type: resource + brief: > + The operating system (OS) on which the process represented by this resource is running. + note: > + In case of virtualized environments, this is the operating system as it is observed by + the process, i.e., the virtualized guest rather than the underlying host. + attributes: + - id: type + type: + allow_custom_values: true + members: + - id: windows + value: 'windows' + brief: "Microsoft Windows" + - id: linux + value: 'linux' + brief: "Linux" + - id: darwin + value: 'darwin' + brief: "Apple Darwin" + - id: freebsd + value: 'freebsd' + brief: "FreeBSD" + - id: netbsd + value: 'netbsd' + brief: "NetBSD" + - id: openbsd + value: 'openbsd' + brief: "OpenBSD" + - id: dragonflybsd + value: 'dragonflybsd' + brief: "DragonFly BSD" + - id: hpux + value: 'hpux' + brief: "HP-UX (Hewlett Packard Unix)" + - id: aix + value: 'aix' + brief: "AIX (Advanced Interactive eXecutive)" + - id: solaris + value: 'solaris' + brief: "SunOS, Oracle Solaris" + - id: z_os + value: 'z_os' + brief: "IBM z/OS" + brief: > + The operating system type. + - id: description + type: string + brief: > + Human readable (not intended to be parsed) OS version information, + like e.g. reported by `ver` or `lsb_release -a` commands. + examples: ['Microsoft Windows [Version 10.0.18363.778]', 'Ubuntu 18.04.1 LTS'] + - id: name + type: string + brief: 'Human readable operating system name.' + examples: ['iOS', 'Android', 'Ubuntu'] + - id: version + type: string + brief: > + The version string of the operating system as defined in + [Version Attributes](/docs/resource/README.md#version-attributes). + examples: ['14.2.1', '18.04.1'] + - id: build_id + type: string + brief: 'Unique identifier for a particular build or compilation of the operating system.' + examples: ['TQ3C.230805.001.B2', '20E247', '22621'] diff --git a/internal/otelschema/_testdata/model/registry/process.yaml b/internal/otelschema/_testdata/model/registry/process.yaml new file mode 100644 index 00000000..dd26c09b --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/process.yaml @@ -0,0 +1,78 @@ +groups: + - id: registry.process + prefix: process + type: resource + brief: > + An operating system process. + attributes: + - id: pid + type: int + brief: > + Process identifier (PID). + examples: [1234] + - id: parent_pid + type: int + brief: > + Parent Process identifier (PID). + examples: [111] + - id: executable.name + type: string + brief: > + The name of the process executable. On Linux based systems, can be set + to the `Name` in `proc/[pid]/status`. On Windows, can be set to the + base name of `GetProcessImageFileNameW`. + examples: ['otelcol'] + - id: executable.path + type: string + brief: > + The full path to the process executable. On Linux based systems, can + be set to the target of `proc/[pid]/exe`. On Windows, can be set to the + result of `GetProcessImageFileNameW`. + examples: ['/usr/bin/cmd/otelcol'] + - id: command + type: string + brief: > + The command used to launch the process (i.e. the command name). On Linux + based systems, can be set to the zeroth string in `proc/[pid]/cmdline`. + On Windows, can be set to the first parameter extracted from `GetCommandLineW`. + examples: ['cmd/otelcol'] + - id: command_line + type: string + brief: > + The full command used to launch the process as a single string representing + the full command. On Windows, can be set to the result of `GetCommandLineW`. + Do not set this if you have to assemble it just for monitoring; use + `process.command_args` instead. + examples: ['C:\cmd\otecol --config="my directory\config.yaml"'] + - id: command_args + type: string[] + brief: > + All the command arguments (including the command/executable itself) as + received by the process. On Linux-based systems (and some other Unixoid + systems supporting procfs), can be set according to the list of + null-delimited strings extracted from `proc/[pid]/cmdline`. For libc-based + executables, this would be the full argv vector passed to `main`. + examples: ['cmd/otecol', '--config=config.yaml'] + - id: owner + type: string + brief: > + The username of the user that owns the process. + examples: 'root' + - id: runtime.name + type: string + brief: > + The name of the runtime of this process. For compiled native binaries, + this SHOULD be the name of the compiler. + examples: ['OpenJDK Runtime Environment'] + - id: runtime.version + type: string + brief: > + The version of the runtime of this process, as returned by the runtime + without modification. + examples: '14.0.2' + - id: runtime.description + type: string + brief: > + An additional description about the runtime of the process, for example + a specific vendor customization of the runtime environment. + examples: 'Eclipse OpenJ9 Eclipse OpenJ9 VM openj9-0.21.0' diff --git a/internal/otelschema/_testdata/model/registry/rpc.yaml b/internal/otelschema/_testdata/model/registry/rpc.yaml new file mode 100644 index 00000000..64f1f9a6 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/rpc.yaml @@ -0,0 +1,190 @@ +groups: + - id: registry.rpc + prefix: rpc + type: attribute_group + brief: 'This document defines attributes for remote procedure calls.' + attributes: + - id: connect_rpc.error_code + type: + members: + - id: cancelled + value: cancelled + - id: unknown + value: unknown + - id: invalid_argument + value: invalid_argument + - id: deadline_exceeded + value: deadline_exceeded + - id: not_found + value: not_found + - id: already_exists + value: already_exists + - id: permission_denied + value: permission_denied + - id: resource_exhausted + value: resource_exhausted + - id: failed_precondition + value: failed_precondition + - id: aborted + value: aborted + - id: out_of_range + value: out_of_range + - id: unimplemented + value: unimplemented + - id: internal + value: internal + - id: unavailable + value: unavailable + - id: data_loss + value: data_loss + - id: unauthenticated + value: unauthenticated + brief: "The [error codes](https://connect.build/docs/protocol/#error-codes) of the Connect request. Error codes are always string values." + - id: connect_rpc.request.metadata + type: template[string[]] + brief: > + Connect request metadata, `` being the normalized Connect Metadata key (lowercase), the value being the metadata values. + note: > + Instrumentations SHOULD require an explicit configuration of which metadata values are to be captured. + Including all request metadata values can be a security risk - explicit configuration helps avoid leaking sensitive information. + examples: ['rpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"]'] + - id: connect_rpc.response.metadata + type: template[string[]] + brief: > + Connect response metadata, `` being the normalized Connect Metadata key (lowercase), the value being the metadata values. + note: > + Instrumentations SHOULD require an explicit configuration of which metadata values are to be captured. + Including all response metadata values can be a security risk - explicit configuration helps avoid leaking sensitive information. + examples: ['rpc.response.metadata.my-custom-metadata-attribute=["attribute_value"]'] + - id: grpc.status_code + type: + members: + - id: ok + brief: OK + value: 0 + - id: cancelled + brief: CANCELLED + value: 1 + - id: unknown + brief: UNKNOWN + value: 2 + - id: invalid_argument + brief: INVALID_ARGUMENT + value: 3 + - id: deadline_exceeded + brief: DEADLINE_EXCEEDED + value: 4 + - id: not_found + brief: NOT_FOUND + value: 5 + - id: already_exists + brief: ALREADY_EXISTS + value: 6 + - id: permission_denied + brief: PERMISSION_DENIED + value: 7 + - id: resource_exhausted + brief: RESOURCE_EXHAUSTED + value: 8 + - id: failed_precondition + brief: FAILED_PRECONDITION + value: 9 + - id: aborted + brief: ABORTED + value: 10 + - id: out_of_range + brief: OUT_OF_RANGE + value: 11 + - id: unimplemented + brief: UNIMPLEMENTED + value: 12 + - id: internal + brief: INTERNAL + value: 13 + - id: unavailable + brief: UNAVAILABLE + value: 14 + - id: data_loss + brief: DATA_LOSS + value: 15 + - id: unauthenticated + brief: UNAUTHENTICATED + value: 16 + brief: "The [numeric status code](https://github.com/grpc/grpc/blob/v1.33.2/doc/statuscodes.md) of the gRPC request." + - id: grpc.request.metadata + type: template[string[]] + brief: > + gRPC request metadata, `` being the normalized gRPC Metadata key (lowercase), the value being the metadata values. + note: > + Instrumentations SHOULD require an explicit configuration of which metadata values are to be captured. + Including all request metadata values can be a security risk - explicit configuration helps avoid leaking sensitive information. + examples: ['rpc.grpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"]'] + - id: grpc.response.metadata + type: template[string[]] + brief: > + gRPC response metadata, `` being the normalized gRPC Metadata key (lowercase), the value being the metadata values. + note: > + Instrumentations SHOULD require an explicit configuration of which metadata values are to be captured. + Including all response metadata values can be a security risk - explicit configuration helps avoid leaking sensitive information. + examples: ['rpc.grpc.response.metadata.my-custom-metadata-attribute=["attribute_value"]'] + - id: jsonrpc.error_code + type: int + brief: "`error.code` property of response if it is an error response." + examples: [-32700, 100] + - id: jsonrpc.error_message + type: string + brief: "`error.message` property of response if it is an error response." + examples: ['Parse error', 'User already exists'] + - id: jsonrpc.request_id + type: string + brief: > + `id` property of request or response. + Since protocol allows id to be int, string, `null` or missing (for notifications), + value is expected to be cast to string for simplicity. + Use empty string in case of `null` value. Omit entirely if this is a notification. + examples: ['10', 'request-7', ''] + - id: jsonrpc.version + type: string + brief: "Protocol version as in `jsonrpc` property of request/response. Since JSON-RPC 1.0 doesn't specify this, the value can be omitted." + examples: ['2.0', '1.0'] + - id: method + type: string + brief: 'The name of the (logical) method being called, must be equal to the $method part in the span name.' + note: > + This is the logical name of the method from the RPC interface perspective, + which can be different from the name of any implementing method/function. + The `code.function` attribute may be used to store the latter + (e.g., method actually executing the call on the server side, + RPC client stub method on the client side). + examples: "exampleMethod" + - id: service + type: string + brief: 'The full (logical) name of the service being called, including its package name, if applicable.' + note: > + This is the logical name of the service from the RPC interface perspective, + which can be different from the name of any implementing class. + The `code.namespace` attribute may be used to store the latter + (despite the attribute name, it may include a class name; + e.g., class with method actually executing the call on the server side, + RPC client stub class on the client side). + examples: "myservice.EchoService" + - id: system + brief: 'A string identifying the remoting system. See below for a list of well-known identifiers.' + type: + allow_custom_values: true + members: + - id: grpc + value: 'grpc' + brief: 'gRPC' + - id: java_rmi + value: 'java_rmi' + brief: 'Java RMI' + - id: dotnet_wcf + value: 'dotnet_wcf' + brief: '.NET WCF' + - id: apache_dubbo + value: 'apache_dubbo' + brief: 'Apache Dubbo' + - id: connect_rpc + value: 'connect_rpc' + brief: 'Connect RPC' diff --git a/internal/otelschema/_testdata/model/registry/server.yaml b/internal/otelschema/_testdata/model/registry/server.yaml new file mode 100644 index 00000000..f47174dd --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/server.yaml @@ -0,0 +1,28 @@ +groups: + - id: server + prefix: server + type: attribute_group + brief: > + These attributes may be used to describe the server in a connection-based network interaction + where there is one side that initiates the connection (the client is the side that initiates the connection). + This covers all TCP network interactions since TCP is connection-based and one side initiates the + connection (an exception is made for peer-to-peer communication over TCP where the "user-facing" surface of the + protocol / API doesn't expose a clear notion of client and server). + This also covers UDP network interactions where one side initiates the interaction, e.g. QUIC (HTTP/3) and DNS. + attributes: + - id: address + stability: stable + type: string + brief: "Server domain name if available without reverse DNS lookup; otherwise, IP address or Unix domain socket name." + note: > + When observed from the client side, and when communicating through an intermediary, `server.address` SHOULD represent + the server address behind any intermediaries, for example proxies, if it's available. + examples: ['example.com', '10.1.2.80', '/tmp/my.sock'] + - id: port + stability: stable + type: int + brief: Server port number. + note: > + When observed from the client side, and when communicating through an intermediary, `server.port` SHOULD represent + the server port behind any intermediaries, for example proxies, if it's available. + examples: [80, 8080, 443] diff --git a/internal/otelschema/_testdata/model/registry/source.yaml b/internal/otelschema/_testdata/model/registry/source.yaml new file mode 100644 index 00000000..263a491c --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/source.yaml @@ -0,0 +1,23 @@ +groups: + - id: source + prefix: source + type: attribute_group + brief: > + These attributes may be used to describe the sender of a network exchange/packet. These should be used + when there is no client/server relationship between the two sides, or when that relationship is unknown. + This covers low-level network interactions (e.g. packet tracing) where you don't know if + there was a connection or which side initiated it. + This also covers unidirectional UDP flows and peer-to-peer communication where the + "user-facing" surface of the protocol / API doesn't expose a clear notion of client and server. + attributes: + - id: address + type: string + brief: "Source address - domain name if available without reverse DNS lookup; otherwise, IP address or Unix domain socket name." + note: > + When observed from the destination side, and when communicating through an intermediary, `source.address` SHOULD represent + the source address behind any intermediaries, for example proxies, if it's available. + examples: ['source.example.com', '10.1.2.80', '/tmp/my.sock'] + - id: port + type: int + brief: 'Source port number' + examples: [3389, 2888] diff --git a/internal/otelschema/_testdata/model/registry/thread.yaml b/internal/otelschema/_testdata/model/registry/thread.yaml new file mode 100644 index 00000000..f6b5b5a5 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/thread.yaml @@ -0,0 +1,17 @@ +groups: + - id: registry.thread + prefix: thread + type: span + brief: > + These attributes may be used for any operation to store information about a thread that started a span. + attributes: + - id: id + type: int + brief: > + Current "managed" thread ID (as opposed to OS thread ID). + examples: 42 + - id: name + type: string + brief: > + Current thread name. + examples: main diff --git a/internal/otelschema/_testdata/model/registry/tls.yaml b/internal/otelschema/_testdata/model/registry/tls.yaml new file mode 100644 index 00000000..421479d1 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/tls.yaml @@ -0,0 +1,165 @@ +groups: + - id: registry.tls + prefix: tls + type: attribute_group + brief: "This document defines semantic convention attributes in the TLS namespace." + attributes: + - id: cipher + brief: > + String indicating the [cipher](https://datatracker.ietf.org/doc/html/rfc5246#appendix-A.5) used during the current connection. + type: string + note: > + The values allowed for `tls.cipher` MUST be one of the `Descriptions` of the + [registered TLS Cipher Suits](https://www.iana.org/assignments/tls-parameters/tls-parameters.xhtml#table-tls-parameters-4). + examples: + [ + "TLS_RSA_WITH_3DES_EDE_CBC_SHA", + "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256", + ] + - id: client.certificate + type: string + brief: > + PEM-encoded stand-alone certificate offered by the client. This is usually mutually-exclusive of `client.certificate_chain` since this value also exists in that list. + examples: ["MII..."] + - id: client.certificate_chain + type: string[] + brief: > + Array of PEM-encoded certificates that make up the certificate chain offered by the client. + This is usually mutually-exclusive of `client.certificate` since that value should be the first certificate in the chain. + examples: ["MII...", "MI..."] + - id: client.hash.md5 + type: string + brief: > + Certificate fingerprint using the MD5 digest of DER-encoded version of certificate offered by the client. + For consistency with other hash values, this value should be formatted as an uppercase hash. + examples: ["0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC"] + - id: client.hash.sha1 + type: string + brief: > + Certificate fingerprint using the SHA1 digest of DER-encoded version of certificate offered by the client. + For consistency with other hash values, this value should be formatted as an uppercase hash. + examples: ["9E393D93138888D288266C2D915214D1D1CCEB2A"] + - id: client.hash.sha256 + type: string + brief: > + Certificate fingerprint using the SHA256 digest of DER-encoded version of certificate offered by the client. + For consistency with other hash values, this value should be formatted as an uppercase hash. + examples: + ["0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0"] + - id: client.issuer + type: string + brief: "Distinguished name of [subject](https://datatracker.ietf.org/doc/html/rfc5280#section-4.1.2.6) of the issuer of the x.509 certificate presented by the client." + examples: + ["CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com"] + - id: client.ja3 + type: string + brief: "A hash that identifies clients based on how they perform an SSL/TLS handshake." + examples: ["d4e5b18d6b55c71272893221c96ba240"] + - id: client.not_after + type: string + brief: "Date/Time indicating when client certificate is no longer considered valid." + examples: ["2021-01-01T00:00:00.000Z"] + - id: client.not_before + type: string + brief: "Date/Time indicating when client certificate is first considered valid." + examples: ["1970-01-01T00:00:00.000Z"] + - id: client.server_name + type: string + brief: "Also called an SNI, this tells the server which hostname to which the client is attempting to connect to." + examples: ["opentelemetry.io"] + - id: client.subject + type: string + brief: "Distinguished name of subject of the x.509 certificate presented by the client." + examples: ["CN=myclient, OU=Documentation Team, DC=example, DC=com"] + - id: client.supported_ciphers + type: string[] + brief: Array of ciphers offered by the client during the client hello. + examples: + [ + '"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", "..."', + ] + - id: curve + brief: "String indicating the curve used for the given cipher, when applicable" + type: string + examples: ["secp256r1"] + - id: established + brief: "Boolean flag indicating if the TLS negotiation was successful and transitioned to an encrypted tunnel." + type: boolean + examples: [true] + - id: next_protocol + brief: > + String indicating the protocol being tunneled. + Per the values in the [IANA registry](https://www.iana.org/assignments/tls-extensiontype-values/tls-extensiontype-values.xhtml#alpn-protocol-ids), + this string should be lower case. + type: string + examples: ["http/1.1"] + - id: protocol.name + brief: > + Normalized lowercase protocol name parsed from original string of the negotiated [SSL/TLS protocol version](https://www.openssl.org/docs/man1.1.1/man3/SSL_get_version.html#RETURN-VALUES) + type: + allow_custom_values: true + members: + - id: ssl + value: ssl + - id: tls + value: tls + - id: protocol.version + brief: > + Numeric part of the version parsed from the original string of the negotiated [SSL/TLS protocol version](https://www.openssl.org/docs/man1.1.1/man3/SSL_get_version.html#RETURN-VALUES) + type: string + examples: ["1.2", "3"] + - id: resumed + brief: "Boolean flag indicating if this TLS connection was resumed from an existing TLS negotiation." + type: boolean + examples: [true] + - id: server.certificate + type: string + brief: > + PEM-encoded stand-alone certificate offered by the server. This is usually mutually-exclusive of `server.certificate_chain` since this value also exists in that list. + examples: ["MII..."] + - id: server.certificate_chain + type: string[] + brief: > + Array of PEM-encoded certificates that make up the certificate chain offered by the server. + This is usually mutually-exclusive of `server.certificate` since that value should be the first certificate in the chain. + examples: ["MII...", "MI..."] + - id: server.hash.md5 + type: string + brief: > + Certificate fingerprint using the MD5 digest of DER-encoded version of certificate offered by the server. + For consistency with other hash values, this value should be formatted as an uppercase hash. + examples: ["0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC"] + - id: server.hash.sha1 + type: string + brief: > + Certificate fingerprint using the SHA1 digest of DER-encoded version of certificate offered by the server. + For consistency with other hash values, this value should be formatted as an uppercase hash. + examples: ["9E393D93138888D288266C2D915214D1D1CCEB2A"] + - id: server.hash.sha256 + type: string + brief: > + Certificate fingerprint using the SHA256 digest of DER-encoded version of certificate offered by the server. + For consistency with other hash values, this value should be formatted as an uppercase hash. + examples: + ["0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0"] + - id: server.issuer + type: string + brief: "Distinguished name of [subject](https://datatracker.ietf.org/doc/html/rfc5280#section-4.1.2.6) of the issuer of the x.509 certificate presented by the client." + examples: + ["CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com"] + - id: server.ja3s + type: string + brief: "A hash that identifies servers based on how they perform an SSL/TLS handshake." + examples: ["d4e5b18d6b55c71272893221c96ba240"] + - id: server.not_after + type: string + brief: "Date/Time indicating when server certificate is no longer considered valid." + examples: ["2021-01-01T00:00:00.000Z"] + - id: server.not_before + type: string + brief: "Date/Time indicating when server certificate is first considered valid." + examples: ["1970-01-01T00:00:00.000Z"] + - id: server.subject + type: string + brief: "Distinguished name of subject of the x.509 certificate presented by the server." + examples: ["CN=myserver, OU=Documentation Team, DC=example, DC=com"] diff --git a/internal/otelschema/_testdata/model/registry/url.yaml b/internal/otelschema/_testdata/model/registry/url.yaml new file mode 100644 index 00000000..985ca912 --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/url.yaml @@ -0,0 +1,41 @@ +groups: + - id: registry.url + brief: Attributes describing URL. + type: attribute_group + prefix: url + attributes: + - id: scheme + stability: stable + type: string + brief: 'The [URI scheme](https://www.rfc-editor.org/rfc/rfc3986#section-3.1) component identifying the used protocol.' + examples: ["https", "ftp", "telnet"] + - id: full + stability: stable + type: string + brief: Absolute URL describing a network resource according to [RFC3986](https://www.rfc-editor.org/rfc/rfc3986) + note: > + For network calls, URL usually has `scheme://host[:port][path][?query][#fragment]` format, where the fragment + is not transmitted over HTTP, but if it is known, it SHOULD be included nevertheless. + + `url.full` MUST NOT contain credentials passed via URL in form of `https://username:password@www.example.com/`. + In such case username and password SHOULD be redacted and attribute's value SHOULD be `https://REDACTED:REDACTED@www.example.com/`. + + `url.full` SHOULD capture the absolute URL when it is available (or can be reconstructed) + and SHOULD NOT be validated or modified except for sanitizing purposes. + examples: ['https://www.foo.bar/search?q=OpenTelemetry#SemConv', '//localhost'] + - id: path + stability: stable + type: string + brief: 'The [URI path](https://www.rfc-editor.org/rfc/rfc3986#section-3.3) component' + examples: ['/search'] + - id: query + stability: stable + type: string + brief: 'The [URI query](https://www.rfc-editor.org/rfc/rfc3986#section-3.4) component' + examples: ["q=OpenTelemetry"] + note: Sensitive content provided in query string SHOULD be scrubbed when instrumentations can identify it. + - id: fragment + stability: stable + type: string + brief: 'The [URI fragment](https://www.rfc-editor.org/rfc/rfc3986#section-3.5) component' + examples: ["SemConv"] diff --git a/internal/otelschema/_testdata/model/registry/user-agent.yaml b/internal/otelschema/_testdata/model/registry/user-agent.yaml new file mode 100644 index 00000000..0fbc5c9f --- /dev/null +++ b/internal/otelschema/_testdata/model/registry/user-agent.yaml @@ -0,0 +1,13 @@ +groups: + - id: registry.user_agent + prefix: user_agent + type: attribute_group + brief: "Describes user-agent attributes." + attributes: + - id: original + stability: stable + type: string + brief: > + Value of the [HTTP User-Agent](https://www.rfc-editor.org/rfc/rfc9110.html#field.user-agent) header sent by the client. + examples: ['CERN-LineMode/2.15 libwww/2.17b3', + 'Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1'] diff --git a/internal/otelschema/_testdata/model/resource/android.yaml b/internal/otelschema/_testdata/model/resource/android.yaml new file mode 100644 index 00000000..dcc236d8 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/android.yaml @@ -0,0 +1,14 @@ +groups: + - id: android + prefix: android + type: resource + brief: > + The Android platform on which the Android application is running. + attributes: + - id: os.api_level + type: string + brief: > + Uniquely identifies the framework API revision offered by a version + (`os.version`) of the android operating system. More information can be found + [here](https://developer.android.com/guide/topics/manifest/uses-sdk-element#ApiLevels). + examples: ['33', '32'] diff --git a/internal/otelschema/_testdata/model/resource/browser.yaml b/internal/otelschema/_testdata/model/resource/browser.yaml new file mode 100644 index 00000000..56830c1d --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/browser.yaml @@ -0,0 +1,56 @@ +groups: + - id: browser + prefix: browser + type: resource + brief: > + The web browser in which the application represented by the resource is running. + The `browser.*` attributes MUST be used only for resources that represent applications + running in a web browser (regardless of whether running on a mobile or desktop device). + attributes: + - id: brands + type: string[] + brief: 'Array of brand name and version separated by a space' + note: > + This value is intended to be taken from the + [UA client hints API](https://wicg.github.io/ua-client-hints/#interface) + (`navigator.userAgentData.brands`). + examples: [" Not A;Brand 99", "Chromium 99", "Chrome 99"] + - id: platform + type: string + brief: 'The platform on which the browser is running' + note: > + This value is intended to be taken from the + [UA client hints API](https://wicg.github.io/ua-client-hints/#interface) + (`navigator.userAgentData.platform`). If unavailable, the legacy + `navigator.platform` API SHOULD NOT be used instead and this attribute + SHOULD be left unset in order for the values to be consistent. + + The list of possible values is defined in the + [W3C User-Agent Client Hints specification](https://wicg.github.io/ua-client-hints/#sec-ch-ua-platform). + Note that some (but not all) of these values can overlap with values + in the [`os.type` and `os.name` attributes](./os.md). + However, for consistency, the values in the `browser.platform` attribute + should capture the exact value that the user agent provides. + examples: ['Windows', 'macOS', 'Android'] + - id: mobile + type: boolean + brief: 'A boolean that is true if the browser is running on a mobile device' + note: > + This value is intended to be taken from the + [UA client hints API](https://wicg.github.io/ua-client-hints/#interface) + (`navigator.userAgentData.mobile`). If unavailable, this attribute + SHOULD be left unset. + - id: language + type: string + brief: 'Preferred language of the user using the browser' + note: > + This value is intended to be taken from the Navigator API + `navigator.language`. + examples: ["en", "en-US", "fr", "fr-FR"] + - ref: user_agent.original + brief: 'Full user-agent string provided by the browser' + note: > + The user-agent value SHOULD be provided only from browsers that do not have a mechanism + to retrieve brands and platform individually from the User-Agent Client Hints API. + To retrieve the value, the legacy `navigator.userAgent` API can be used. + examples: ['Mozilla/5.0 (Macintosh; Intel Mac OS X 10_15_7) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/95.0.4638.54 Safari/537.36'] diff --git a/internal/otelschema/_testdata/model/resource/cloud.yaml b/internal/otelschema/_testdata/model/resource/cloud.yaml new file mode 100644 index 00000000..4699f848 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/cloud.yaml @@ -0,0 +1,13 @@ +groups: + - id: cloud + prefix: cloud + type: resource + brief: > + A cloud environment (e.g. GCP, Azure, AWS) + attributes: + - ref: cloud.provider + - ref: cloud.account.id + - ref: cloud.region + - ref: cloud.resource_id + - ref: cloud.availability_zone + - ref: cloud.platform diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/ecs.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/ecs.yaml new file mode 100644 index 00000000..2c6b8c9b --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/ecs.yaml @@ -0,0 +1,42 @@ +groups: + - id: aws.ecs + prefix: aws.ecs + type: resource + brief: > + Resources used by AWS Elastic Container Service (ECS). + attributes: + - id: container.arn + type: string + brief: > + The Amazon Resource Name (ARN) of an [ECS container instance](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/ECS_instances.html). + examples: ['arn:aws:ecs:us-west-1:123456789123:container/32624152-9086-4f0e-acae-1a75b14fe4d9'] + - id: cluster.arn + type: string + brief: > + The ARN of an [ECS cluster](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/clusters.html). + examples: ['arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster'] + - id: launchtype + type: + allow_custom_values: false + members: + - id: ec2 + value: "ec2" + - id: fargate + value: "fargate" + brief: > + The [launch type](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/launch_types.html) for an ECS task. + - id: task.arn + type: string + brief: > + The ARN of an [ECS task definition](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task_definitions.html). + examples: ['arn:aws:ecs:us-west-1:123456789123:task/10838bed-421f-43ef-870a-f43feacbbb5b'] + - id: task.family + type: string + brief: > + The task definition family this task definition is a member of. + examples: ['opentelemetry-family'] + - id: task.revision + type: string + brief: > + The revision for this task definition. + examples: ["8", "26"] diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/eks.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/eks.yaml new file mode 100644 index 00000000..2c897253 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/eks.yaml @@ -0,0 +1,12 @@ +groups: + - id: aws.eks + prefix: aws.eks + type: resource + brief: > + Resources used by AWS Elastic Kubernetes Service (EKS). + attributes: + - id: cluster.arn + type: string + brief: > + The ARN of an EKS cluster. + examples: ['arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster'] diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/logs.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/logs.yaml new file mode 100644 index 00000000..8a433629 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/logs.yaml @@ -0,0 +1,39 @@ +groups: + - id: aws.log + prefix: aws.log + type: resource + brief: > + Resources specific to Amazon Web Services. + attributes: + - id: group.names + type: string[] + brief: > + The name(s) of the AWS log group(s) an application is writing to. + examples: ['/aws/lambda/my-function', 'opentelemetry-service'] + note: > + Multiple log groups must be supported for cases like multi-container applications, + where a single application has sidecar containers, and each write to their own log + group. + - id: group.arns + type: string[] + brief: > + The Amazon Resource Name(s) (ARN) of the AWS log group(s). + examples: ['arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:*'] + note: > + See the + [log group ARN format documentation](https://docs.aws.amazon.com/AmazonCloudWatch/latest/logs/iam-access-control-overview-cwl.html#CWL_ARN_Format). + - id: stream.names + type: string[] + brief: > + The name(s) of the AWS log stream(s) an application is writing to. + examples: ['logs/main/10838bed-421f-43ef-870a-f43feacbbb5b'] + - id: stream.arns + type: string[] + brief: > + The ARN(s) of the AWS log stream(s). + examples: ['arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:log-stream:logs/main/10838bed-421f-43ef-870a-f43feacbbb5b'] + note: > + See the + [log stream ARN format documentation](https://docs.aws.amazon.com/AmazonCloudWatch/latest/logs/iam-access-control-overview-cwl.html#CWL_ARN_Format). + One log group can contain several log streams, so these ARNs necessarily identify both a log + group and a log stream. diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/cloud_run.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/cloud_run.yaml new file mode 100644 index 00000000..e4da8f59 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/cloud_run.yaml @@ -0,0 +1,23 @@ +groups: + - id: gcp.cloud_run + prefix: gcp.cloud_run + type: resource + brief: > + Resource used by Google Cloud Run. + attributes: + - id: job.execution + type: string + brief: > + The name of the Cloud Run + [execution](https://cloud.google.com/run/docs/managing/job-executions) + being run for the Job, as set by the + [`CLOUD_RUN_EXECUTION`](https://cloud.google.com/run/docs/container-contract#jobs-env-vars) + environment variable. + examples: ['job-name-xxxx', 'sample-job-mdw84'] + - id: job.task_index + type: int + brief: > + The index for a task within an execution as provided by the + [`CLOUD_RUN_TASK_INDEX`](https://cloud.google.com/run/docs/container-contract#jobs-env-vars) + environment variable. + examples: [0, 1] diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/gce.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/gce.yaml new file mode 100644 index 00000000..879d0ea3 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/gce.yaml @@ -0,0 +1,22 @@ +groups: + - id: gcp.gce + prefix: gcp.gce + type: resource + brief: > + Resources used by Google Compute Engine (GCE). + attributes: + - id: instance.name + type: string + brief: > + The instance name of a GCE instance. This is the value + provided by `host.name`, the visible name of the instance in + the Cloud Console UI, and the prefix for the default + hostname of the instance as defined by the [default internal + DNS + name](https://cloud.google.com/compute/docs/internal-dns#instance-fully-qualified-domain-names). + examples: ['instance-1', 'my-vm-name'] + - id: instance.hostname + type: string + brief: > + The hostname of a GCE instance. This is the full value of the default or [custom hostname](https://cloud.google.com/compute/docs/instances/custom-hostname-vm). + examples: ['my-host1234.example.com', 'sample-vm.us-west1-b.c.my-project.internal'] diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/heroku.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/heroku.yaml new file mode 100644 index 00000000..e73eddc1 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/cloud_provider/heroku.yaml @@ -0,0 +1,25 @@ +groups: + - id: heroku + prefix: heroku + type: resource + brief: > + Heroku dyno metadata + attributes: + - id: release.creation_timestamp + type: string + brief: > + Time and date the release was created + examples: [ '2022-10-23T18:00:42Z' ] + requirement_level: opt_in + - id: release.commit + type: string + brief: > + Commit hash for the current release + examples: [ 'e6134959463efd8966b20e75b913cafe3f5ec' ] + requirement_level: opt_in + - id: app.id + type: string + brief: > + Unique identifier for the application + examples: [ '2daa2797-e42b-4624-9322-ec3f968df4da' ] + requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/resource/container.yaml b/internal/otelschema/_testdata/model/resource/container.yaml new file mode 100644 index 00000000..97923e44 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/container.yaml @@ -0,0 +1,22 @@ +groups: + - id: container + prefix: container + type: resource + brief: > + A container instance. + attributes: + - ref: container.name + - ref: container.id + - ref: container.runtime + - ref: container.image.name + - ref: container.image.tags + - ref: container.image.id + - ref: container.image.repo_digests + - ref: container.command + requirement_level: opt_in + - ref: container.command_line + requirement_level: opt_in + - ref: container.command_args + requirement_level: opt_in + - ref: container.labels + - ref: oci.manifest.digest diff --git a/internal/otelschema/_testdata/model/resource/deployment_environment.yaml b/internal/otelschema/_testdata/model/resource/deployment_environment.yaml new file mode 100644 index 00000000..7ada3f97 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/deployment_environment.yaml @@ -0,0 +1,21 @@ +groups: + - id: deployment + prefix: deployment + type: resource + brief: > + The software deployment. + attributes: + - id: environment + type: string + brief: > + Name of the [deployment environment](https://wikipedia.org/wiki/Deployment_environment) + (aka deployment tier). + note: | + `deployment.environment` does not affect the uniqueness constraints defined through + the `service.namespace`, `service.name` and `service.instance.id` resource attributes. + This implies that resources carrying the following attribute combinations MUST be + considered to be identifying the same service: + + * `service.name=frontend`, `deployment.environment=production` + * `service.name=frontend`, `deployment.environment=staging`. + examples: ['staging', 'production'] diff --git a/internal/otelschema/_testdata/model/resource/device.yaml b/internal/otelschema/_testdata/model/resource/device.yaml new file mode 100644 index 00000000..8fab02a8 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/device.yaml @@ -0,0 +1,11 @@ +groups: + - id: device + prefix: device + type: resource + brief: > + The device on which the process represented by this resource is running. + attributes: + - ref: device.id + - ref: device.manufacturer + - ref: device.model.identifier + - ref: device.model.name diff --git a/internal/otelschema/_testdata/model/resource/faas.yaml b/internal/otelschema/_testdata/model/resource/faas.yaml new file mode 100644 index 00000000..7f1a02e0 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/faas.yaml @@ -0,0 +1,63 @@ +groups: + - id: faas_resource + prefix: faas + type: resource + brief: > + A serverless instance. + attributes: + - id: name + type: string + requirement_level: required + brief: > + The name of the single function that this runtime instance executes. + note: | + This is the name of the function as configured/deployed on the FaaS + platform and is usually different from the name of the callback + function (which may be stored in the + [`code.namespace`/`code.function`](/docs/general/attributes.md#source-code-attributes) + span attributes). + + For some cloud providers, the above definition is ambiguous. The following + definition of function name MUST be used for this attribute + (and consequently the span name) for the listed cloud providers/products: + + * **Azure:** The full name `/`, i.e., function app name + followed by a forward slash followed by the function name (this form + can also be seen in the resource JSON for the function). + This means that a span attribute MUST be used, as an Azure function + app can host multiple functions that would usually share + a TracerProvider (see also the `cloud.resource_id` attribute). + examples: ['my-function', 'myazurefunctionapp/some-function-name'] + - id: version + type: string + brief: The immutable version of the function being executed. + note: | + Depending on the cloud provider and platform, use: + + * **AWS Lambda:** The [function version](https://docs.aws.amazon.com/lambda/latest/dg/configuration-versions.html) + (an integer represented as a decimal string). + * **Google Cloud Run (Services):** The [revision](https://cloud.google.com/run/docs/managing/revisions) + (i.e., the function name plus the revision suffix). + * **Google Cloud Functions:** The value of the + [`K_REVISION` environment variable](https://cloud.google.com/functions/docs/env-var#runtime_environment_variables_set_automatically). + * **Azure Functions:** Not applicable. Do not set this attribute. + examples: ['26', 'pinkfroid-00002'] + - id: instance + type: string + brief: > + The execution environment ID as a string, that will be potentially reused + for other invocations to the same function/function version. + note: > + * **AWS Lambda:** Use the (full) log stream name. + examples: ['2021/06/28/[$LATEST]2f399eb14537447da05ab2a2e39309de'] + - id: max_memory + type: int + brief: > + The amount of memory available to the serverless function converted to Bytes. + note: > + It's recommended to set this attribute since e.g. too little memory can easily + stop a Java AWS Lambda function from working correctly. + On AWS Lambda, the environment variable `AWS_LAMBDA_FUNCTION_MEMORY_SIZE` + provides this information (which must be multiplied by 1,048,576). + examples: 134217728 + - ref: cloud.resource_id diff --git a/internal/otelschema/_testdata/model/resource/host.yaml b/internal/otelschema/_testdata/model/resource/host.yaml new file mode 100644 index 00000000..b90b90dc --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/host.yaml @@ -0,0 +1,37 @@ +groups: + - id: host + prefix: host + type: resource + brief: > + A host is defined as a computing instance. For example, physical servers, virtual machines, switches or disk array. + attributes: + - ref: host.id + - ref: host.name + - ref: host.type + - ref: host.arch + - ref: host.image.name + - ref: host.image.id + - ref: host.image.version + - ref: host.ip + requirement_level: opt_in + - ref: host.mac + requirement_level: opt_in + + - id: host.cpu + prefix: host.cpu + type: resource + brief: > + A host's CPU information + attributes: + - ref: host.cpu.vendor.id + requirement_level: opt_in + - ref: host.cpu.family + requirement_level: opt_in + - ref: host.cpu.model.id + requirement_level: opt_in + - ref: host.cpu.model.name + requirement_level: opt_in + - ref: host.cpu.stepping + requirement_level: opt_in + - ref: host.cpu.cache.l2.size + requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/resource/k8s.yaml b/internal/otelschema/_testdata/model/resource/k8s.yaml new file mode 100644 index 00000000..20947c8a --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/k8s.yaml @@ -0,0 +1,98 @@ +groups: + - id: k8s.cluster + prefix: k8s.cluster + type: resource + brief: > + A Kubernetes Cluster. + attributes: + - ref: k8s.cluster.name + - ref: k8s.cluster.uid + + - id: k8s.node + prefix: k8s.node + type: resource + brief: > + A Kubernetes Node object. + attributes: + - ref: k8s.node.name + - ref: k8s.node.uid + + - id: k8s.namespace + prefix: k8s.namespace + type: resource + brief: > + A Kubernetes Namespace. + attributes: + - ref: k8s.namespace.name + + - id: k8s.pod + prefix: k8s.pod + type: resource + brief: > + A Kubernetes Pod object. + attributes: + - ref: k8s.pod.uid + - ref: k8s.pod.name + + - id: k8s.container + prefix: k8s.container + type: resource + brief: > + A container in a [PodTemplate](https://kubernetes.io/docs/concepts/workloads/pods/#pod-templates). + attributes: + - ref: k8s.container.name + - ref: k8s.container.restart_count + + - id: k8s.replicaset + prefix: k8s.replicaset + type: resource + brief: > + A Kubernetes ReplicaSet object. + attributes: + - ref: k8s.replicaset.uid + - ref: k8s.replicaset.name + + - id: k8s.deployment + prefix: k8s.deployment + type: resource + brief: > + A Kubernetes Deployment object. + attributes: + - ref: k8s.deployment.uid + - ref: k8s.deployment.name + + - id: k8s.statefulset + prefix: k8s.statefulset + type: resource + brief: > + A Kubernetes StatefulSet object. + attributes: + - ref: k8s.statefulset.uid + - ref: k8s.statefulset.name + + - id: k8s.daemonset + prefix: k8s.daemonset + type: resource + brief: > + A Kubernetes DaemonSet object. + attributes: + - ref: k8s.daemonset.uid + - ref: k8s.daemonset.name + + - id: k8s.job + prefix: k8s.job + type: resource + brief: > + A Kubernetes Job object. + attributes: + - ref: k8s.job.uid + - ref: k8s.job.name + + - id: k8s.cronjob + prefix: k8s.cronjob + type: resource + brief: > + A Kubernetes CronJob object. + attributes: + - ref: k8s.cronjob.uid + - ref: k8s.cronjob.name diff --git a/internal/otelschema/_testdata/model/resource/os.yaml b/internal/otelschema/_testdata/model/resource/os.yaml new file mode 100644 index 00000000..772fdde3 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/os.yaml @@ -0,0 +1,16 @@ +groups: + - id: os + prefix: os + type: resource + brief: > + The operating system (OS) on which the process represented by this resource is running. + note: > + In case of virtualized environments, this is the operating system as it is observed by + the process, i.e., the virtualized guest rather than the underlying host. + attributes: + - ref: os.type + requirement_level: required + - ref: os.description + - ref: os.name + - ref: os.version + - ref: os.build_id diff --git a/internal/otelschema/_testdata/model/resource/process.yaml b/internal/otelschema/_testdata/model/resource/process.yaml new file mode 100644 index 00000000..61223d61 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/process.yaml @@ -0,0 +1,42 @@ +groups: + - id: process + prefix: process + type: resource + brief: > + An operating system process. + attributes: + - ref: process.pid + - ref: process.parent_pid + - ref: process.executable.name + requirement_level: + conditionally_required: See alternative attributes below. + - ref: process.executable.path + requirement_level: + conditionally_required: See alternative attributes below. + - ref: process.command + requirement_level: + conditionally_required: See alternative attributes below. + - ref: process.command_line + requirement_level: + conditionally_required: See alternative attributes below. + - ref: process.command_args + requirement_level: + conditionally_required: See alternative attributes below. + - ref: process.owner + constraints: + - any_of: + - process.executable.name + - process.executable.path + - process.command + - process.command_line + - process.command_args + + - id: process.runtime + prefix: process.runtime + type: resource + brief: > + The single (language) runtime instance which is monitored. + attributes: + - ref: process.runtime.name + - ref: process.runtime.version + - ref: process.runtime.description diff --git a/internal/otelschema/_testdata/model/resource/service.yaml b/internal/otelschema/_testdata/model/resource/service.yaml new file mode 100644 index 00000000..e930b621 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/service.yaml @@ -0,0 +1,23 @@ +groups: + - id: service + prefix: service + type: resource + brief: > + A service instance. + attributes: + - id: name + type: string + requirement_level: required + brief: > + Logical name of the service. + note: > + MUST be the same for all instances of horizontally scaled services. + If the value was not specified, SDKs MUST fallback to `unknown_service:` concatenated + with [`process.executable.name`](process.md#process), e.g. `unknown_service:bash`. + If `process.executable.name` is not available, the value MUST be set to `unknown_service`. + examples: ["shoppingcart"] + - id: version + type: string + brief: > + The version string of the service API or implementation. The format is not defined by these conventions. + examples: ["2.0.0", "a01dbef8a"] diff --git a/internal/otelschema/_testdata/model/resource/service_experimental.yaml b/internal/otelschema/_testdata/model/resource/service_experimental.yaml new file mode 100644 index 00000000..43c869ee --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/service_experimental.yaml @@ -0,0 +1,37 @@ +groups: + - id: service_experimental + prefix: service + type: resource + brief: > + A service instance. + attributes: + - id: namespace + type: string + brief: > + A namespace for `service.name`. + note: > + A string value having a meaning that helps to distinguish a group of services, + for example the team name that owns a group of services. + `service.name` is expected to be unique within the same namespace. + If `service.namespace` is not specified in the Resource then `service.name` + is expected to be unique for all services that have no explicit namespace defined + (so the empty/unspecified namespace is simply one more valid namespace). + Zero-length namespace string is assumed equal to unspecified namespace. + examples: ["Shop"] + - id: instance.id + type: string + brief: > + The string ID of the service instance. + note: > + MUST be unique for each instance of the same `service.namespace,service.name` pair + (in other words `service.namespace,service.name,service.instance.id` triplet MUST be globally unique). + The ID helps to distinguish instances of the same service that exist at the same time + (e.g. instances of a horizontally scaled service). It is preferable for the ID to be persistent + and stay the same for the lifetime of the service instance, however it is acceptable that + the ID is ephemeral and changes during important lifetime events for the service + (e.g. service restarts). + If the service has no inherent unique ID that can be used as the value of this attribute + it is recommended to generate a random Version 1 or Version 4 RFC 4122 UUID + (services aiming for reproducible UUIDs may also use Version 5, see RFC 4122 + for more recommendations). + examples: ["my-k8s-pod-deployment-1", "627cc493-f310-47de-96bd-71410b7dec09"] diff --git a/internal/otelschema/_testdata/model/resource/telemetry.yaml b/internal/otelschema/_testdata/model/resource/telemetry.yaml new file mode 100644 index 00000000..6966b4a8 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/telemetry.yaml @@ -0,0 +1,57 @@ +groups: + - id: telemetry + prefix: telemetry + type: resource + brief: > + The telemetry SDK used to capture data recorded by the instrumentation libraries. + attributes: + - id: sdk.name + type: string + requirement_level: required + brief: > + The name of the telemetry SDK as defined above. + note: | + The OpenTelemetry SDK MUST set the `telemetry.sdk.name` attribute to `opentelemetry`. + If another SDK, like a fork or a vendor-provided implementation, is used, this SDK MUST set the + `telemetry.sdk.name` attribute to the fully-qualified class or module name of this SDK's main entry point + or another suitable identifier depending on the language. + The identifier `opentelemetry` is reserved and MUST NOT be used in this case. + All custom identifiers SHOULD be stable across different versions of an implementation. + examples: ["opentelemetry"] + - id: sdk.language + type: + allow_custom_values: true + members: + - id: cpp + value: "cpp" + - id: dotnet + value: "dotnet" + - id: erlang + value: "erlang" + - id: go + value: "go" + - id: java + value: "java" + - id: nodejs + value: "nodejs" + - id: php + value: "php" + - id: python + value: "python" + - id: ruby + value: "ruby" + - id: rust + value: "rust" + - id: swift + value: "swift" + - id: webjs + value: "webjs" + requirement_level: required + brief: > + The language of the telemetry SDK. + - id: sdk.version + type: string + requirement_level: required + brief: > + The version string of the telemetry SDK. + examples: ["1.2.3"] diff --git a/internal/otelschema/_testdata/model/resource/telemetry_experimental.yaml b/internal/otelschema/_testdata/model/resource/telemetry_experimental.yaml new file mode 100644 index 00000000..8f7b2355 --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/telemetry_experimental.yaml @@ -0,0 +1,20 @@ +groups: + - id: telemetry_experimental + prefix: telemetry + type: resource + brief: > + The telemetry SDK used to capture data recorded by the instrumentation libraries. + attributes: + - id: distro.name + type: string + brief: > + The name of the auto instrumentation agent or distribution, if used. + note: | + Official auto instrumentation agents and distributions SHOULD set the `telemetry.distro.name` attribute to + a string starting with `opentelemetry-`, e.g. `opentelemetry-java-instrumentation`. + examples: ["parts-unlimited-java"] + - id: distro.version + type: string + brief: > + The version string of the auto instrumentation agent or distribution, if used. + examples: ["1.2.3"] diff --git a/internal/otelschema/_testdata/model/resource/webengine.yaml b/internal/otelschema/_testdata/model/resource/webengine.yaml new file mode 100644 index 00000000..5e0cdf8f --- /dev/null +++ b/internal/otelschema/_testdata/model/resource/webengine.yaml @@ -0,0 +1,23 @@ +groups: + - id: webengine_resource + prefix: webengine + type: resource + brief: > + Resource describing the packaged software running the application code. Web engines are typically executed using process.runtime. + attributes: + - id: name + type: string + requirement_level: required + brief: > + The name of the web engine. + examples: ['WildFly'] + - id: version + type: string + brief: > + The version of the web engine. + examples: ['21.0.0'] + - id: description + type: string + brief: > + Additional description of the web engine (e.g. detailed version and edition information). + examples: ['WildFly Full 21.0.0.Final (WildFly Core 13.0.1.Final) - 2.2.2.Final'] diff --git a/internal/otelschema/_testdata/model/scope/exporter/exporter.yaml b/internal/otelschema/_testdata/model/scope/exporter/exporter.yaml new file mode 100644 index 00000000..3bbf3fe0 --- /dev/null +++ b/internal/otelschema/_testdata/model/scope/exporter/exporter.yaml @@ -0,0 +1,30 @@ +groups: + - id: otel.scope + prefix: otel.scope + type: resource + brief: Attributes used by non-OTLP exporters to represent OpenTelemetry Scope's concepts. + attributes: + - id: name + type: string + brief: The name of the instrumentation scope - (`InstrumentationScope.Name` in OTLP). + examples: ['io.opentelemetry.contrib.mongodb'] + - id: version + type: string + brief: The version of the instrumentation scope - (`InstrumentationScope.Version` in OTLP). + examples: ['1.0.0'] + - id: otel.library + prefix: otel.library + type: resource + brief: > + Span attributes used by non-OTLP exporters to represent OpenTelemetry Scope's concepts. + attributes: + - id: name + type: string + stability: "deprecated" + brief: Deprecated, use the `otel.scope.name` attribute. + examples: ['io.opentelemetry.contrib.mongodb'] + - id: version + type: string + stability: "deprecated" + brief: Deprecated, use the `otel.scope.version` attribute. + examples: ['1.0.0'] diff --git a/internal/otelschema/_testdata/model/session.yaml b/internal/otelschema/_testdata/model/session.yaml new file mode 100644 index 00000000..f2212653 --- /dev/null +++ b/internal/otelschema/_testdata/model/session.yaml @@ -0,0 +1,26 @@ +groups: + - id: session-id + prefix: session + type: attribute_group + brief: > + Session is defined as the period of time encompassing all activities performed by the application and the actions + executed by the end user. + + Consequently, a Session is represented as a collection of Logs, Events, and Spans emitted by the Client Application + throughout the Session's duration. Each Session is assigned a unique identifier, which is included as an attribute in + the Logs, Events, and Spans generated during the Session's lifecycle. + + When a session reaches end of life, typically due to user inactivity or session timeout, a new session identifier + will be assigned. The previous session identifier may be provided by the instrumentation so that telemetry + backends can link the two sessions. + attributes: + - id: id + type: string + brief: "A unique id to identify a session." + examples: "00112233-4455-6677-8899-aabbccddeeff" + requirement_level: opt_in + - id: previous_id + type: string + brief: "The previous `session.id` for this user, when known." + examples: "00112233-4455-6677-8899-aabbccddeeff" + requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/trace/aws/lambda.yaml b/internal/otelschema/_testdata/model/trace/aws/lambda.yaml new file mode 100644 index 00000000..73e77ea7 --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/aws/lambda.yaml @@ -0,0 +1,14 @@ +groups: + - id: aws.lambda + prefix: aws.lambda + type: span + brief: > + Span attributes used by AWS Lambda (in addition to general `faas` attributes). + attributes: + - id: invoked_arn + type: string + brief: > + The full invoked ARN as provided on the `Context` passed to the function + (`Lambda-Runtime-Invoked-Function-Arn` header on the `/runtime/invocation/next` applicable). + note: This may be different from `cloud.resource_id` if an alias is involved. + examples: ['arn:aws:lambda:us-east-1:123456:function:myfunction:myalias'] diff --git a/internal/otelschema/_testdata/model/trace/cloudevents.yaml b/internal/otelschema/_testdata/model/trace/cloudevents.yaml new file mode 100644 index 00000000..6fcf7ae2 --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/cloudevents.yaml @@ -0,0 +1,36 @@ +groups: + - id: cloudevents + prefix: cloudevents + type: span + brief: > + This document defines attributes for CloudEvents. + CloudEvents is a specification on how to define event data in a standard way. + These attributes can be attached to spans when performing operations with CloudEvents, regardless of the protocol being used. + attributes: + - id: event_id + type: string + requirement_level: required + brief: > + The [event_id](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#id) uniquely identifies the event. + examples: ['123e4567-e89b-12d3-a456-426614174000', '0001'] + - id: event_source + type: string + requirement_level: required + brief: > + The [source](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#source-1) identifies the context in which an event happened. + examples: ['https://github.com/cloudevents', '/cloudevents/spec/pull/123', 'my-service' ] + - id: event_spec_version + type: string + brief: > + The [version of the CloudEvents specification](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#specversion) which the event uses. + examples: '1.0' + - id: event_type + type: string + brief: > + The [event_type](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#type) contains a value describing the type of event related to the originating occurrence. + examples: ['com.github.pull_request.opened', 'com.example.object.deleted.v2'] + - id: event_subject + type: string + brief: > + The [subject](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#subject) of the event in the context of the event producer (identified by source). + examples: 'mynewfile.jpg' diff --git a/internal/otelschema/_testdata/model/trace/compatibility.yaml b/internal/otelschema/_testdata/model/trace/compatibility.yaml new file mode 100644 index 00000000..2e3fe12d --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/compatibility.yaml @@ -0,0 +1,21 @@ +groups: + - id: opentracing + prefix: opentracing + type: span + brief: 'This document defines semantic conventions for the OpenTracing Shim' + note: > + These conventions are used by the OpenTracing Shim layer. + attributes: + - id: ref_type + brief: 'Parent-child Reference type' + note: > + The causal relationship between a child Span and a parent Span. + type: + allow_custom_values: false + members: + - id: child_of + value: 'child_of' + brief: "The parent Span depends on the child Span in some capacity" + - id: follows_from + value: 'follows_from' + brief: "The parent Span doesn't depend in any way on the result of the child Span" diff --git a/internal/otelschema/_testdata/model/trace/database.yaml b/internal/otelschema/_testdata/model/trace/database.yaml new file mode 100644 index 00000000..ed4f5716 --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/database.yaml @@ -0,0 +1,259 @@ +groups: + - id: db + type: span + brief: > + This document defines the attributes used to perform database client calls. + span_kind: client + attributes: + - ref: db.system + tag: connection-level + requirement_level: required + + - ref: db.connection_string + tag: connection-level + - ref: db.user + tag: connection-level + - ref: db.jdbc.driver_classname + tag: connection-level-tech-specific + - ref: db.name + tag: call-level + requirement_level: + conditionally_required: If applicable. + - ref: db.statement + tag: call-level + requirement_level: + recommended: > + Should be collected by default only if there is sanitization that excludes sensitive information. + - ref: db.operation + tag: call-level + requirement_level: + conditionally_required: If `db.statement` is not applicable. + - ref: server.address + tag: connection-level + brief: > + Name of the database host. + - ref: server.port + tag: connection-level + requirement_level: + conditionally_required: If using a port other than the default port for this DBMS and if `server.address` is set. + - ref: network.peer.address + tag: connection-level + - ref: network.peer.port + requirement_level: + recommended: If `network.peer.address` is set. + tag: connection-level + - ref: network.transport + tag: connection-level + - ref: network.type + tag: connection-level + + - id: db.mssql + type: span + extends: db + brief: > + Connection-level attributes for Microsoft SQL Server + attributes: + - ref: db.mssql.instance_name + tag: connection-level-tech-specific + + - id: db.cassandra + type: span + extends: db + brief: > + Call-level attributes for Cassandra + attributes: + - ref: db.name + tag: call-level-tech-specific-cassandra + brief: > + The keyspace name in Cassandra. + examples: ["mykeyspace"] + note: For Cassandra the `db.name` should be set to the Cassandra keyspace name. + - ref: db.cassandra.page_size + tag: call-level-tech-specific-cassandra + - ref: db.cassandra.consistency_level + tag: call-level-tech-specific-cassandra + - ref: db.cassandra.table + tag: call-level-tech-specific-cassandra + - ref: db.cassandra.idempotence + tag: call-level-tech-specific-cassandra + - ref: db.cassandra.speculative_execution_count + tag: call-level-tech-specific-cassandra + - ref: db.cassandra.coordinator.id + tag: call-level-tech-specific-cassandra + - ref: db.cassandra.coordinator.dc + tag: call-level-tech-specific-cassandra + + - id: db.hbase + type: span + extends: db + brief: > + Call-level attributes for HBase + attributes: + - ref: db.name + tag: call-level-tech-specific + brief: > + The HBase namespace. + examples: ['mynamespace'] + note: For HBase the `db.name` should be set to the HBase namespace. + + - id: db.couchdb + type: span + extends: db + brief: > + Call-level attributes for CouchDB + attributes: + - ref: db.operation + tag: call-level-tech-specific + brief: > + The HTTP method + the target REST route. + examples: ['GET /{db}/{docid}'] + note: > + In **CouchDB**, `db.operation` should be set to the HTTP method + + the target REST route according to the API reference documentation. + For example, when retrieving a document, `db.operation` would be set to + (literally, i.e., without replacing the placeholders with concrete values): + [`GET /{db}/{docid}`](http://docs.couchdb.org/en/stable/api/document/common.html#get--db-docid). + + - id: db.redis + type: span + extends: db + brief: > + Call-level attributes for Redis + attributes: + - ref: db.redis.database_index + requirement_level: + conditionally_required: If other than the default database (`0`). + tag: call-level-tech-specific + - ref: db.statement + tag: call-level-tech-specific + brief: > + The full syntax of the Redis CLI command. + examples: ["HMSET myhash field1 'Hello' field2 'World'"] + note: > + For **Redis**, the value provided for `db.statement` SHOULD correspond to the syntax of the Redis CLI. + If, for example, the [`HMSET` command](https://redis.io/commands/hmset) is invoked, `"HMSET myhash field1 'Hello' field2 'World'"` would be a suitable value for `db.statement`. + + - id: db.mongodb + type: span + extends: db + brief: > + Call-level attributes for MongoDB + attributes: + - ref: db.mongodb.collection + requirement_level: required + tag: call-level-tech-specific + + - id: db.elasticsearch + type: span + extends: db + brief: > + Call-level attributes for Elasticsearch + attributes: + - ref: http.request.method + requirement_level: required + tag: call-level-tech-specific + - ref: db.operation + requirement_level: required + brief: The endpoint identifier for the request. + examples: [ 'search', 'ml.close_job', 'cat.aliases' ] + tag: call-level-tech-specific + - ref: url.full + requirement_level: required + examples: [ 'https://localhost:9200/index/_search?q=user.id:kimchy' ] + tag: call-level-tech-specific + - ref: db.statement + requirement_level: + recommended: > + Should be collected by default for search-type queries and only if there is sanitization that excludes + sensitive information. + brief: The request body for a [search-type query](https://www.elastic.co/guide/en/elasticsearch/reference/current/search.html), as a json string. + examples: [ '"{\"query\":{\"term\":{\"user.id\":\"kimchy\"}}}"' ] + tag: call-level-tech-specific + - ref: server.address + tag: call-level-tech-specific + - ref: server.port + tag: call-level-tech-specific + - ref: db.elasticsearch.cluster.name + requirement_level: + recommended: > + When communicating with an Elastic Cloud deployment, this should be collected from the "X-Found-Handling-Cluster" HTTP response header. + tag: call-level-tech-specific + - ref: db.elasticsearch.node.name + requirement_level: + recommended: > + When communicating with an Elastic Cloud deployment, this should be collected from the "X-Found-Handling-Instance" HTTP response header. + tag: call-level-tech-specific + - ref: db.elasticsearch.path_parts + requirement_level: + conditionally_required: when the url has dynamic values + tag: call-level-tech-specific + + - id: db.sql + type: span + extends: 'db' + brief: > + Call-level attributes for SQL databases + attributes: + - ref: db.sql.table + tag: call-level-tech-specific + + - id: db.cosmosdb + type: span + extends: db + prefix: db.cosmosdb + brief: > + Call-level attributes for Cosmos DB. + attributes: + - ref: db.cosmosdb.client_id + tag: call-level-tech-specific + - ref: db.cosmosdb.operation_type + requirement_level: + conditionally_required: when performing one of the operations in this list + tag: call-level-tech-specific + - ref: user_agent.original + brief: 'Full user-agent string is generated by Cosmos DB SDK' + note: > + The user-agent value is generated by SDK which is a combination of
+ `sdk_version` : Current version of SDK. e.g. 'cosmos-netstandard-sdk/3.23.0'
+ `direct_pkg_version` : Direct package version used by Cosmos DB SDK. e.g. '3.23.1'
+ `number_of_client_instances` : Number of cosmos client instances created by the application. e.g. '1'
+ `type_of_machine_architecture` : Machine architecture. e.g. 'X64'
+ `operating_system` : Operating System. e.g. 'Linux 5.4.0-1098-azure 104 18'
+ `runtime_framework` : Runtime Framework. e.g. '.NET Core 3.1.32'
+ `failover_information` : Generated key to determine if region failover enabled. + Format Reg-{D (Disabled discovery)}-S(application region)|L(List of preferred regions)|N(None, user did not configure it). + Default value is "NS". + examples: ['cosmos-netstandard-sdk/3.23.0\|3.23.1\|1\|X64\|Linux 5.4.0-1098-azure 104 18\|.NET Core 3.1.32\|S\|'] + tag: call-level-tech-specific + - ref: db.cosmosdb.connection_mode + requirement_level: + conditionally_required: if not `direct` (or pick gw as default) + tag: call-level-tech-specific + - ref: db.cosmosdb.container + requirement_level: + conditionally_required: if available + tag: call-level-tech-specific + - ref: db.cosmosdb.request_content_length + tag: call-level-tech-specific + - ref: db.cosmosdb.status_code + requirement_level: + conditionally_required: if response was received + tag: call-level-tech-specific + - ref: db.cosmosdb.sub_status_code + requirement_level: + conditionally_required: when response was received and contained sub-code. + tag: call-level-tech-specific + - ref: db.cosmosdb.request_charge + requirement_level: + conditionally_required: when available + tag: call-level-tech-specific + + - id: db.tech + type: span + brief: "Semantic convention group for specific technologies" + constraints: + - include: 'db.cassandra' + - include: 'db.redis' + - include: 'db.mongodb' + - include: 'db.sql' + - include: 'db.cosmosdb' diff --git a/internal/otelschema/_testdata/model/trace/exporter/exporter.yaml b/internal/otelschema/_testdata/model/trace/exporter/exporter.yaml new file mode 100644 index 00000000..2989475d --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/exporter/exporter.yaml @@ -0,0 +1,21 @@ +groups: + - id: otel_span + prefix: otel + type: span + brief: Span attributes used by non-OTLP exporters to represent OpenTelemetry Span's concepts. + attributes: + - id: status_code + type: + allow_custom_values: false + members: + - id: ok + value: OK + brief: 'The operation has been validated by an Application developer or Operator to have completed successfully.' + - id: error + value: ERROR + brief: 'The operation contains an error.' + brief: Name of the code, either "OK" or "ERROR". MUST NOT be set if the status code is UNSET. + - id: status_description + type: string + brief: "Description of the Status if it has a value, otherwise not set." + examples: ['resource not found'] diff --git a/internal/otelschema/_testdata/model/trace/faas.yaml b/internal/otelschema/_testdata/model/trace/faas.yaml new file mode 100644 index 00000000..84b7141b --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/faas.yaml @@ -0,0 +1,144 @@ +groups: + - id: faas_span + prefix: faas + type: span + brief: > + This semantic convention describes an instance of a function that + runs without provisioning or managing of servers (also known as + serverless functions or Function as a Service (FaaS)) with spans. + attributes: + - ref: faas.trigger + note: | + For the server/consumer span on the incoming side, + `faas.trigger` MUST be set. + + Clients invoking FaaS instances usually cannot set `faas.trigger`, + since they would typically need to look in the payload to determine + the event type. If clients set it, it should be the same as the + trigger that corresponding incoming would have (i.e., this has + nothing to do with the underlying transport used to make the API + call to invoke the lambda, which is often HTTP). + - id: invocation_id + type: string + brief: 'The invocation ID of the current function invocation.' + examples: 'af9d5aa4-a685-4c5f-a22b-444f80b3cc28' + - ref: cloud.resource_id + + - id: faas_span.datasource + prefix: faas.document + type: span + brief: > + Semantic Convention for FaaS triggered as a response to some data + source operation such as a database or filesystem read/write. + attributes: + - id: collection + type: string + requirement_level: required + brief: > + The name of the source on which the triggering operation was performed. + For example, in Cloud Storage or S3 corresponds to the bucket name, + and in Cosmos DB to the database name. + examples: ['myBucketName', 'myDbName'] + - id: operation + requirement_level: required + type: + allow_custom_values: true + members: + - id: insert + value: 'insert' + brief: 'When a new object is created.' + - id: edit + value: 'edit' + brief: 'When an object is modified.' + - id: delete + value: 'delete' + brief: 'When an object is deleted.' + brief: 'Describes the type of the operation that was performed on the data.' + - id: time + type: string + brief: > + A string containing the time when the data was accessed in the + [ISO 8601](https://www.iso.org/iso-8601-date-and-time-format.html) + format expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). + examples: "2020-01-23T13:47:06Z" + - id: name + type: string + brief: > + The document name/table subjected to the operation. + For example, in Cloud Storage or S3 is the name of + the file, and in Cosmos DB the table name. + examples: ["myFile.txt", "myTableName"] + + - id: faas_span.http + type: span + brief: > + Semantic Convention for FaaS triggered as a response to some data + source operation such as a database or filesystem read/write. + constraints: + - include: trace.http.server + attributes: [] + + - id: faas_span.pubsub + type: span + brief: > + Semantic Convention for FaaS set to be executed when messages are + sent to a messaging system. + constraints: + - include: messaging + attributes: [] + + - id: faas_span.timer + prefix: faas + type: span + brief: > + Semantic Convention for FaaS scheduled to be executed regularly. + attributes: + - id: time + type: string + brief: > + A string containing the function invocation time in the + [ISO 8601](https://www.iso.org/iso-8601-date-and-time-format.html) + format expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). + examples: "2020-01-23T13:47:06Z" + - id: cron + type: string + brief: > + A string containing the schedule period as + [Cron Expression](https://docs.oracle.com/cd/E12058_01/doc/doc.1014/e12030/cron_expressions.htm). + examples: "0/5 * * * ? *" + + - id: faas_span.in + span_kind: server + prefix: faas + type: span + brief: > + Contains additional attributes for incoming FaaS spans. + attributes: + - id: coldstart + type: boolean + brief: > + A boolean that is true if the serverless function is executed for the + first time (aka cold-start). + - ref: faas.trigger + requirement_level: required + note: | + For the server/consumer span on the incoming side, + `faas.trigger` MUST be set. + + Clients invoking FaaS instances usually cannot set `faas.trigger`, + since they would typically need to look in the payload to determine + the event type. If clients set it, it should be the same as the + trigger that corresponding incoming would have (i.e., this has + nothing to do with the underlying transport used to make the API + call to invoke the lambda, which is often HTTP). + + - id: faas_span.out + span_kind: client + prefix: faas + type: span + brief: > + Contains additional attributes for outgoing FaaS spans. + attributes: + - ref: faas.invoked_name + - ref: faas.invoked_provider + - ref: faas.invoked_region diff --git a/internal/otelschema/_testdata/model/trace/feature-flag.yaml b/internal/otelschema/_testdata/model/trace/feature-flag.yaml new file mode 100644 index 00000000..50706ba2 --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/feature-flag.yaml @@ -0,0 +1,34 @@ +groups: + - id: feature_flag + prefix: feature_flag + type: event + brief: > + This semantic convention defines the attributes used to represent a + feature flag evaluation as an event. + attributes: + - id: key + type: string + requirement_level: required + brief: The unique identifier of the feature flag. + examples: ["logo-color"] + - id: provider_name + type: string + requirement_level: recommended + brief: The name of the service provider that performs the flag evaluation. + examples: ["Flag Manager"] + - id: variant + type: string + requirement_level: recommended + examples: ["red", "true", "on"] + brief: > + SHOULD be a semantic identifier for a value. If one is unavailable, a + stringified version of the value can be used. + note: |- + A semantic identifier, commonly referred to as a variant, provides a means + for referring to a value without including the value itself. This can + provide additional context for understanding the meaning behind a value. + For example, the variant `red` maybe be used for the value `#c05543`. + + A stringified version of the value can be used in situations where a + semantic identifier is unavailable. String representation of the value + should be determined by the implementer. diff --git a/internal/otelschema/_testdata/model/trace/http.yaml b/internal/otelschema/_testdata/model/trace/http.yaml new file mode 100644 index 00000000..1e5667ad --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/http.yaml @@ -0,0 +1,91 @@ +groups: + - id: trace.http.common + extends: attributes.http.common + type: attribute_group + brief: 'This document defines semantic conventions for HTTP client and server Spans.' + note: > + These conventions can be used for http and https schemes + and various HTTP versions like 1.1, 2 and SPDY. + attributes: + - ref: http.request.method_original + requirement_level: + conditionally_required: If and only if it's different than `http.request.method`. + - ref: http.response.header + requirement_level: opt_in + - ref: http.request.method + sampling_relevant: true + requirement_level: required + - ref: network.peer.address + - ref: network.peer.port + requirement_level: + recommended: If `network.peer.address` is set. + - ref: network.transport + requirement_level: opt_in + note: > + Generally `tcp` for `HTTP/1.0`, `HTTP/1.1`, and `HTTP/2`. Generally `udp` for `HTTP/3`. + Other obscure implementations are possible. + + - id: trace.http.client + type: span + extends: attributes.http.client + span_kind: client + brief: 'Semantic Convention for HTTP Client' + attributes: + - ref: http.request.resend_count + requirement_level: + recommended: if and only if request was retried. + - ref: http.request.header + requirement_level: opt_in + - ref: server.address + sampling_relevant: true + - ref: server.port + sampling_relevant: true + - ref: url.full + sampling_relevant: true + requirement_level: required + - ref: user_agent.original + requirement_level: opt_in + - ref: url.scheme + + - id: trace.http.server + type: span + extends: attributes.http.server + span_kind: server + brief: 'Semantic Convention for HTTP Server' + attributes: + - ref: http.route + - ref: http.request.header + sampling_relevant: true + requirement_level: opt_in + - ref: server.address + sampling_relevant: true + - ref: server.port + sampling_relevant: true + - ref: network.local.address + requirement_level: opt_in + brief: Local socket address. Useful in case of a multi-IP host. + - ref: network.local.port + requirement_level: opt_in + brief: Local socket port. Useful in case of a multi-port host. + - ref: client.address + sampling_relevant: true + note: > + The IP address of the original client behind all proxies, if + known (e.g. from [Forwarded#for](https://developer.mozilla.org/docs/Web/HTTP/Headers/Forwarded#for), + [X-Forwarded-For](https://developer.mozilla.org/docs/Web/HTTP/Headers/X-Forwarded-For), or a similar header). + Otherwise, the immediate client peer address. + examples: ['83.164.160.102'] + - ref: client.port + requirement_level: opt_in + brief: The port of whichever client was captured in `client.address`. + - ref: url.path + requirement_level: required + sampling_relevant: true + - ref: url.query + requirement_level: + conditionally_required: If and only if one was received/sent. + sampling_relevant: true + - ref: url.scheme + sampling_relevant: true + - ref: user_agent.original + sampling_relevant: true diff --git a/internal/otelschema/_testdata/model/trace/instrumentation/aws-sdk.yml b/internal/otelschema/_testdata/model/trace/instrumentation/aws-sdk.yml new file mode 100644 index 00000000..90b6d1c9 --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/instrumentation/aws-sdk.yml @@ -0,0 +1,516 @@ +groups: + - id: aws + prefix: aws + type: span + brief: > + The `aws` conventions apply to operations using the AWS SDK. They map request or response parameters + in AWS SDK API calls to attributes on a Span. The conventions have been collected over time based + on feedback from AWS users of tracing and will continue to evolve as new interesting conventions + are found. + + Some descriptions are also provided for populating general OpenTelemetry semantic conventions based on + these APIs. + attributes: + - ref: rpc.system + brief: "The value `aws-api`." + requirement_level: required + examples: + - aws-api + - ref: rpc.service + brief: "The name of the service to which a request is made, as returned by the AWS SDK." + examples: + - DynamoDB + - S3 + - ref: rpc.method + brief: "The name of the operation corresponding to the request, as returned by the AWS SDK" + examples: + - GetItem + - PutItem + - id: request_id + type: string + brief: "The AWS request ID as returned in the response headers `x-amz-request-id` or `x-amz-requestid`." + examples: + - 79b9da39-b7ae-508a-a6bc-864b2829c622 + - C9ER4AJX75574TDJ + + - id: dynamodb.all + type: span + brief: "Attributes always filled for all DynamoDB request types." + attributes: + - ref: db.system + brief: "The value `dynamodb`." + requirement_level: required + examples: + - dynamodb + + - id: dynamodb.shared + extends: aws + prefix: aws.dynamodb + type: span + brief: "Attributes that exist for multiple DynamoDB request types." + attributes: + - ref: db.operation + brief: "The same value as `rpc.method`." + examples: + - GetItem + - PutItem + - id: table_names + type: string[] + brief: The keys in the `RequestItems` object field. + examples: + - Users + - Cats + - id: consumed_capacity + type: string[] + brief: "The JSON-serialized value of each item in the `ConsumedCapacity` response field." + examples: + - '{ + "CapacityUnits": number, + "GlobalSecondaryIndexes": { + "string" : { + "CapacityUnits": number, + "ReadCapacityUnits": number, + "WriteCapacityUnits": number + } + }, + "LocalSecondaryIndexes": { + "string" : { + "CapacityUnits": number, + "ReadCapacityUnits": number, + "WriteCapacityUnits": number + } + }, + "ReadCapacityUnits": number, + "Table": { + "CapacityUnits": number, + "ReadCapacityUnits": number, + "WriteCapacityUnits": number + }, + "TableName": "string", + "WriteCapacityUnits": number + }' + - id: item_collection_metrics + type: string + brief: "The JSON-serialized value of the `ItemCollectionMetrics` response field." + examples: + - '{ + "string" : [ + { + "ItemCollectionKey": { + "string" : { + "B": blob, + "BOOL": boolean, + "BS": [ blob ], + "L": [ + "AttributeValue" + ], + "M": { + "string" : "AttributeValue" + }, + "N": "string", + "NS": [ "string" ], + "NULL": boolean, + "S": "string", + "SS": [ "string" ] + } + }, + "SizeEstimateRangeGB": [ number ] + } + ] + }' + - id: provisioned_read_capacity + type: double + brief: "The value of the `ProvisionedThroughput.ReadCapacityUnits` request parameter." + examples: + - 1.0 + - 2.0 + - id: provisioned_write_capacity + type: double + brief: "The value of the `ProvisionedThroughput.WriteCapacityUnits` request parameter." + examples: + - 1.0 + - 2.0 + - id: consistent_read + type: boolean + brief: "The value of the `ConsistentRead` request parameter." + - id: projection + type: string + brief: "The value of the `ProjectionExpression` request parameter." + examples: + - Title + - Title, Price, Color + - Title, Description, RelatedItems, ProductReviews + - id: limit + type: int + brief: "The value of the `Limit` request parameter." + examples: + - 10 + - id: attributes_to_get + type: string[] + brief: "The value of the `AttributesToGet` request parameter." + examples: + - lives + - id + - id: index_name + type: string + brief: "The value of the `IndexName` request parameter." + examples: + - name_to_group + - id: select + type: string + brief: "The value of the `Select` request parameter." + examples: + - ALL_ATTRIBUTES + - COUNT + + - id: dynamodb.batchgetitem + brief: DynamoDB.BatchGetItem + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - ref: aws.dynamodb.table_names + - ref: aws.dynamodb.consumed_capacity + + - id: dynamodb.batchwriteitem + brief: DynamoDB.BatchWriteItem + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - ref: aws.dynamodb.table_names + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.item_collection_metrics + + - id: dynamodb.createtable + brief: DynamoDB.CreateTable + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - id: global_secondary_indexes + type: string[] + brief: "The JSON-serialized value of each item of the `GlobalSecondaryIndexes` request field" + examples: + - '{ + "IndexName": "string", + "KeySchema": [ + { + "AttributeName": "string", + "KeyType": "string" + } + ], + "Projection": { + "NonKeyAttributes": [ "string" ], + "ProjectionType": "string" + }, + "ProvisionedThroughput": { + "ReadCapacityUnits": number, + "WriteCapacityUnits": number + } + }' + - id: local_secondary_indexes + type: string[] + brief: "The JSON-serialized value of each item of the `LocalSecondaryIndexes` request field." + examples: + - '{ + "IndexArn": "string", + "IndexName": "string", + "IndexSizeBytes": number, + "ItemCount": number, + "KeySchema": [ + { + "AttributeName": "string", + "KeyType": "string" + } + ], + "Projection": { + "NonKeyAttributes": [ "string" ], + "ProjectionType": "string" + } + }' + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.item_collection_metrics + - ref: aws.dynamodb.provisioned_read_capacity + - ref: aws.dynamodb.provisioned_write_capacity + + - id: dynamodb.deleteitem + brief: DynamoDB.DeleteItem + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.item_collection_metrics + + - id: dynamodb.deletetable + brief: DynamoDB.DeleteTable + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + + - id: dynamodb.describetable + brief: DynamoDB.DescribeTable + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + + - id: dynamodb.getitem + brief: DynamoDB.GetItem + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.consistent_read + - ref: aws.dynamodb.projection + + - id: dynamodb.listtables + brief: DynamoDB.ListTables + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - id: exclusive_start_table + type: string + brief: "The value of the `ExclusiveStartTableName` request parameter." + examples: + - Users + - CatsTable + - id: table_count + type: int + brief: "The the number of items in the `TableNames` response parameter." + examples: + - 20 + - ref: aws.dynamodb.limit + + - id: dynamodb.putitem + brief: DynamoDB.PutItem + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - ref: aws.dynamodb.table_names + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.item_collection_metrics + + - id: dynamodb.query + brief: DynamoDB.Query + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - id: scan_forward + type: boolean + brief: "The value of the `ScanIndexForward` request parameter." + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.consistent_read + - ref: aws.dynamodb.limit + - ref: aws.dynamodb.projection + - ref: aws.dynamodb.attributes_to_get + - ref: aws.dynamodb.index_name + - ref: aws.dynamodb.select + + - id: dynamodb.scan + brief: DynamoDB.Scan + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - id: segment + type: int + brief: "The value of the `Segment` request parameter." + examples: + - 10 + - id: total_segments + type: int + brief: "The value of the `TotalSegments` request parameter." + examples: + - 100 + - id: count + type: int + brief: "The value of the `Count` response parameter." + examples: + - 10 + - id: scanned_count + type: int + brief: "The value of the `ScannedCount` response parameter." + examples: + - 50 + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.consistent_read + - ref: aws.dynamodb.limit + - ref: aws.dynamodb.projection + - ref: aws.dynamodb.attributes_to_get + - ref: aws.dynamodb.index_name + - ref: aws.dynamodb.select + + - id: dynamodb.updateitem + brief: DynamoDB.UpdateItem + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.item_collection_metrics + + - id: dynamodb.updatetable + brief: DynamoDB.UpdateTable + extends: aws + prefix: aws.dynamodb + type: span + attributes: + - id: attribute_definitions + type: string[] + brief: "The JSON-serialized value of each item in the `AttributeDefinitions` request field." + examples: + - '{ + "AttributeName": "string", + "AttributeType": "string" + }' + - id: global_secondary_index_updates + type: string[] + brief: "The JSON-serialized value of each item in the the `GlobalSecondaryIndexUpdates` request field." + examples: + - '{ + "Create": { + "IndexName": "string", + "KeySchema": [ + { + "AttributeName": "string", + "KeyType": "string" + } + ], + "Projection": { + "NonKeyAttributes": [ "string" ], + "ProjectionType": "string" + }, + "ProvisionedThroughput": { + "ReadCapacityUnits": number, + "WriteCapacityUnits": number + } + }' + - ref: aws.dynamodb.table_names + brief: "A single-element array with the value of the TableName request parameter." + examples: + - Users + - ref: aws.dynamodb.consumed_capacity + - ref: aws.dynamodb.provisioned_read_capacity + - ref: aws.dynamodb.provisioned_write_capacity + + - id: aws.s3 + extends: aws + prefix: aws.s3 + type: span + brief: "Attributes that exist for S3 request types." + attributes: + - id: bucket + type: string + brief: "The S3 bucket name the request refers to. Corresponds to the `--bucket` parameter of the [S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/index.html) operations." + examples: + - some-bucket-name + note: | + The `bucket` attribute is applicable to all S3 operations that reference a bucket, i.e. that require the bucket name as a mandatory parameter. + This applies to almost all S3 operations except `list-buckets`. + - id: key + type: string + brief: "The S3 object key the request refers to. Corresponds to the `--key` parameter of the [S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/index.html) operations." + examples: + - someFile.yml + note: | + The `key` attribute is applicable to all object-related S3 operations, i.e. that require the object key as a mandatory parameter. + This applies in particular to the following operations: + + - [copy-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/copy-object.html) + - [delete-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/delete-object.html) + - [get-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/get-object.html) + - [head-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/head-object.html) + - [put-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/put-object.html) + - [restore-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/restore-object.html) + - [select-object-content](https://docs.aws.amazon.com/cli/latest/reference/s3api/select-object-content.html) + - [abort-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/abort-multipart-upload.html) + - [complete-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/complete-multipart-upload.html) + - [create-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/create-multipart-upload.html) + - [list-parts](https://docs.aws.amazon.com/cli/latest/reference/s3api/list-parts.html) + - [upload-part](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part.html) + - [upload-part-copy](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part-copy.html) + - id: copy_source + type: string + brief: "The source object (in the form `bucket`/`key`) for the copy operation." + examples: + - someFile.yml + note: | + The `copy_source` attribute applies to S3 copy operations and corresponds to the `--copy-source` parameter + of the [copy-object operation within the S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/copy-object.html). + This applies in particular to the following operations: + + - [copy-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/copy-object.html) + - [upload-part-copy](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part-copy.html) + - id: upload_id + type: string + brief: "Upload ID that identifies the multipart upload." + examples: + - 'dfRtDYWFbkRONycy.Yxwh66Yjlx.cph0gtNBtJ' + note: | + The `upload_id` attribute applies to S3 multipart-upload operations and corresponds to the `--upload-id` parameter + of the [S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/index.html) multipart operations. + This applies in particular to the following operations: + + - [abort-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/abort-multipart-upload.html) + - [complete-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/complete-multipart-upload.html) + - [list-parts](https://docs.aws.amazon.com/cli/latest/reference/s3api/list-parts.html) + - [upload-part](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part.html) + - [upload-part-copy](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part-copy.html) + - id: delete + type: string + brief: "The delete request container that specifies the objects to be deleted." + examples: + - 'Objects=[{Key=string,VersionId=string},{Key=string,VersionId=string}],Quiet=boolean' + note: | + The `delete` attribute is only applicable to the [delete-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/delete-object.html) operation. + The `delete` attribute corresponds to the `--delete` parameter of the + [delete-objects operation within the S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/delete-objects.html). + - id: part_number + type: int + brief: "The part number of the part being uploaded in a multipart-upload operation. This is a positive integer between 1 and 10,000." + examples: + - 3456 + note: | + The `part_number` attribute is only applicable to the [upload-part](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part.html) + and [upload-part-copy](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part-copy.html) operations. + The `part_number` attribute corresponds to the `--part-number` parameter of the + [upload-part operation within the S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part.html). diff --git a/internal/otelschema/_testdata/model/trace/instrumentation/graphql.yml b/internal/otelschema/_testdata/model/trace/instrumentation/graphql.yml new file mode 100644 index 00000000..88af65e9 --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/instrumentation/graphql.yml @@ -0,0 +1,32 @@ +groups: + - id: graphql + prefix: graphql + type: span + brief: > + This document defines semantic conventions to apply when instrumenting the GraphQL implementation. They map + GraphQL operations to attributes on a Span. + attributes: + - id: operation.name + brief: "The name of the operation being executed." + type: string + examples: 'findBookById' + - id: operation.type + brief: "The type of the operation being executed." + type: + allow_custom_values: false + members: + - id: query + value: "query" + brief: "GraphQL query" + - id: mutation + value: "mutation" + brief: "GraphQL mutation" + - id: subscription + value: "subscription" + brief: "GraphQL subscription" + examples: ['query', 'mutation', 'subscription'] + - id: document + brief: "The GraphQL document being executed." + type: string + note: The value may be sanitized to exclude sensitive information. + examples: 'query findBookById { bookById(id: ?) { name } }' diff --git a/internal/otelschema/_testdata/model/trace/messaging.yaml b/internal/otelschema/_testdata/model/trace/messaging.yaml new file mode 100644 index 00000000..9163de0b --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/messaging.yaml @@ -0,0 +1,182 @@ +groups: + - id: messaging.message + type: attribute_group + brief: 'Semantic convention describing per-message attributes populated on messaging spans or links.' + attributes: + - ref: messaging.destination.name + - ref: messaging.message.id + - ref: messaging.message.conversation_id + - ref: messaging.message.envelope.size + - ref: messaging.message.body.size + + - id: messaging.destination + type: attribute_group + brief: 'Semantic convention for attributes that describe messaging destination on broker' + note: | + Destination attributes should be set on publish, receive, or other spans + describing messaging operations. + + Destination attributes should be set when the messaging operation handles + single messages. When the operation handles a batch of messages, + the destination attributes should only be applied when the attribute value + applies to all messages in the batch. + In other cases, destination attributes may be set on links. + attributes: + - ref: messaging.destination.name + - ref: messaging.destination.template + - ref: messaging.destination.temporary + - ref: messaging.destination.anonymous + + - id: messaging.destination_publish + prefix: messaging.destination_publish + type: attribute_group + brief: > + Semantic convention for attributes that describe the publish messaging destination on broker. + The term Publish Destination refers to the destination the message was originally published to. + These attributes should be used on the consumer side when information about + the publish destination is available and different than the destination message are consumed from. + note: | + Publish destination attributes should be set on publish, receive, + or other spans describing messaging operations. + Destination attributes should be set when the messaging operation handles + single messages. When the operation handles a batch of messages, + the destination attributes should only be applied when the attribute value + applies to all messages in the batch. + In other cases, destination attributes may be set on links. + attributes: + - ref: messaging.destination_publish.name + - ref: messaging.destination_publish.anonymous + + - id: messaging + type: span + brief: > + This document defines general attributes used in + messaging systems. + attributes: + - ref: messaging.system + requirement_level: required + - ref: messaging.operation + requirement_level: required + - ref: messaging.batch.message_count + requirement_level: + conditionally_required: If the span describes an operation on a batch of messages. + - ref: messaging.client_id + requirement_level: + recommended: If a client id is available + - ref: messaging.destination.name + requirement_level: + conditionally_required: If span describes operation on a single message or if the value applies to all messages in the batch. + - ref: messaging.destination.template + requirement_level: + conditionally_required: > + If available. Instrumentations MUST NOT use `messaging.destination.name` as template + unless low-cardinality of destination name is guaranteed. + - ref: messaging.destination.temporary + requirement_level: + conditionally_required: If value is `true`. When missing, the value is assumed to be `false`. + - ref: messaging.destination.anonymous + requirement_level: + conditionally_required: If value is `true`. When missing, the value is assumed to be `false`. + - ref: messaging.message.id + requirement_level: + recommended: Only for spans that represent an operation on a single message. + - ref: messaging.message.conversation_id + requirement_level: + recommended: Only if span represents operation on a single message. + - ref: messaging.message.envelope.size + requirement_level: + recommended: Only if span represents operation on a single message. + - ref: messaging.message.body.size + requirement_level: + recommended: Only if span represents operation on a single message. + - ref: server.address + note: > + This should be the IP/hostname of the broker (or other network-level peer) this specific message is sent to/received from. + requirement_level: + conditionally_required: If available. + - ref: network.peer.address + tag: connection-level + - ref: network.peer.port + requirement_level: + recommended: If `network.peer.address` is set. + tag: connection-level + - ref: network.transport + tag: connection-level + - ref: network.type + tag: connection-level + - ref: network.protocol.name + examples: ['amqp', 'mqtt'] + - ref: network.protocol.version + + - id: messaging.rabbitmq + type: attribute_group + extends: messaging + brief: > + Attributes for RabbitMQ + attributes: + - ref: messaging.rabbitmq.destination.routing_key + requirement_level: + conditionally_required: If not empty. + tag: tech-specific-rabbitmq + + - id: messaging.kafka + type: attribute_group + extends: messaging + brief: > + Attributes for Apache Kafka + attributes: + - ref: messaging.kafka.message.key + tag: tech-specific-kafka + - ref: messaging.kafka.consumer.group + tag: tech-specific-kafka + - ref: messaging.kafka.destination.partition + tag: tech-specific-kafka + - ref: messaging.kafka.message.offset + tag: tech-specific-kafka + - ref: messaging.kafka.message.tombstone + requirement_level: + conditionally_required: If value is `true`. When missing, the value is assumed to be `false`. + tag: tech-specific-kafka + + - id: messaging.rocketmq + type: attribute_group + extends: messaging + brief: > + Attributes for Apache RocketMQ + attributes: + - ref: messaging.rocketmq.namespace + requirement_level: required + tag: tech-specific-rocketmq + - ref: messaging.rocketmq.client_group + requirement_level: required + tag: tech-specific-rocketmq + - ref: messaging.rocketmq.message.delivery_timestamp + requirement_level: + conditionally_required: If the message type is delay and delay time level is not specified. + tag: tech-specific-rocketmq + - ref: messaging.rocketmq.message.delay_time_level + requirement_level: + conditionally_required: If the message type is delay and delivery timestamp is not specified. + tag: tech-specific-rocketmq + - ref: messaging.rocketmq.message.group + requirement_level: + conditionally_required: If the message type is FIFO. + tag: tech-specific-rocketmq + - ref: messaging.rocketmq.message.type + tag: tech-specific-rocketmq + - ref: messaging.rocketmq.message.tag + tag: tech-specific-rocketmq + - ref: messaging.rocketmq.message.keys + tag: tech-specific-rocketmq + - ref: messaging.rocketmq.consumption_model + tag: tech-specific-rocketmq + - id: messaging.gcp_pubsub + type: attribute_group + extends: messaging + brief: > + Attributes for Google Cloud Pub/Sub + attributes: + - ref: messaging.gcp_pubsub.message.ordering_key + tag: tech-specific-gcp-pubsub + requirement_level: + conditionally_required: If the message type has an ordering key set. diff --git a/internal/otelschema/_testdata/model/trace/rpc.yaml b/internal/otelschema/_testdata/model/trace/rpc.yaml new file mode 100644 index 00000000..de36d676 --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/rpc.yaml @@ -0,0 +1,129 @@ +groups: + - id: rpc + prefix: rpc + type: span + brief: 'This document defines semantic conventions for remote procedure calls.' + events: [rpc.message] + attributes: + - ref: rpc.system + requirement_level: required + - ref: rpc.service + - ref: rpc.method + - ref: network.transport + - ref: network.type + - ref: server.address + requirement_level: required + brief: > + RPC server [host name](https://grpc.github.io/grpc/core/md_doc_naming.html). + note: > + May contain server IP address, DNS name, or local socket name. When host component is an IP address, + instrumentations SHOULD NOT do a reverse proxy lookup to obtain DNS name and SHOULD set + `server.address` to the IP address provided in the host component. + - ref: server.port + requirement_level: + conditionally_required: if the port is supported by the network transport used for communication. + + - id: rpc.client + type: span + brief: 'This document defines semantic conventions for remote procedure call client spans.' + extends: rpc + attributes: + - ref: network.peer.address + - ref: network.peer.port + requirement_level: + recommended: If `network.peer.address` is set. + + - id: rpc.server + type: span + extends: rpc + span_kind: server + brief: 'Semantic Convention for RPC server spans' + attributes: + - ref: client.address + - ref: client.port + - ref: network.peer.address + - ref: network.peer.port + requirement_level: + recommended: If `network.peer.address` is set. + - ref: network.transport + - ref: network.type + + - id: rpc.grpc + type: span + extends: rpc + brief: 'Tech-specific attributes for gRPC.' + attributes: + - ref: rpc.grpc.status_code + tag: grpc-tech-specific + requirement_level: required + - ref: rpc.grpc.request.metadata + tag: grpc-tech-specific + requirement_level: opt_in + - ref: rpc.grpc.response.metadata + tag: grpc-tech-specific + requirement_level: opt_in + + - id: rpc.jsonrpc + prefix: rpc.jsonrpc + type: span + extends: rpc + brief: 'Tech-specific attributes for [JSON RPC](https://www.jsonrpc.org/).' + attributes: + - ref: rpc.jsonrpc.version + tag: jsonrpc-tech-specific + requirement_level: + conditionally_required: If other than the default version (`1.0`) + - ref: rpc.jsonrpc.request_id + tag: jsonrpc-tech-specific + - ref: rpc.jsonrpc.error_code + tag: jsonrpc-tech-specific + requirement_level: + conditionally_required: If response is not successful. + - ref: rpc.jsonrpc.error_message + tag: jsonrpc-tech-specific + - ref: rpc.method + tag: jsonrpc-tech-specific + requirement_level: required + note: > + This is always required for jsonrpc. See the note in the general + RPC conventions for more information. + + - id: rpc.message + prefix: "message" # TODO: Change the prefix to rpc.message? + type: event + brief: "RPC received/sent message." + attributes: + - id: type + type: + members: + - id: sent + value: "SENT" + - id: received + value: "RECEIVED" + brief: "Whether this is a received or sent message." + - id: id + type: int + brief: "MUST be calculated as two different counters starting from `1` one for sent messages and one for received message." + note: "This way we guarantee that the values will be consistent between different implementations." + - id: compressed_size + type: int + brief: "Compressed size of the message in bytes." + - id: uncompressed_size + type: int + brief: "Uncompressed size of the message in bytes." + + - id: rpc.connect_rpc + type: span + extends: rpc + brief: 'Tech-specific attributes for Connect RPC.' + attributes: + - ref: rpc.connect_rpc.error_code + tag: connect_rpc-tech-specific + requirement_level: + conditionally_required: If response is not successful and if error code available. + - ref: rpc.connect_rpc.request.metadata + tag: connect_rpc-tech-specific + requirement_level: opt_in + - ref: rpc.connect_rpc.response.metadata + tag: connect_rpc-tech-specific + requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/trace/trace-exception.yaml b/internal/otelschema/_testdata/model/trace/trace-exception.yaml new file mode 100644 index 00000000..a11082eb --- /dev/null +++ b/internal/otelschema/_testdata/model/trace/trace-exception.yaml @@ -0,0 +1,38 @@ +groups: + - id: trace-exception + prefix: exception + type: event + brief: > + This document defines the attributes used to + report a single exception associated with a span. + attributes: + - ref: exception.type + - ref: exception.message + - ref: exception.stacktrace + - id: escaped + type: boolean + brief: > + SHOULD be set to true if the exception event is recorded at a point where + it is known that the exception is escaping the scope of the span. + note: |- + An exception is considered to have escaped (or left) the scope of a span, + if that span is ended while the exception is still logically "in flight". + This may be actually "in flight" in some languages (e.g. if the exception + is passed to a Context manager's `__exit__` method in Python) but will + usually be caught at the point of recording the exception in most languages. + + It is usually not possible to determine at the point where an exception is thrown + whether it will escape the scope of a span. + However, it is trivial to know that an exception + will escape, if one checks for an active exception just before ending the span, + as done in the [example above](#recording-an-exception). + + It follows that an exception may still escape the scope of the span + even if the `exception.escaped` attribute was not set or set to false, + since the event might have been recorded at a time where it was not + clear whether the exception will escape. + + constraints: + - any_of: + - "exception.type" + - "exception.message" diff --git a/internal/otelschema/_testdata/model/url.yaml b/internal/otelschema/_testdata/model/url.yaml new file mode 100644 index 00000000..bb27dc4e --- /dev/null +++ b/internal/otelschema/_testdata/model/url.yaml @@ -0,0 +1,13 @@ +groups: + - id: url + brief: Attributes describing URL. + type: attribute_group + prefix: url + attributes: + - ref: url.scheme + - ref: url.full + tag: sensitive-information + - ref: url.path + - ref: url.query + tag: sensitive-information + - ref: url.fragment diff --git a/internal/otelschema/_testdata/model/version.properties b/internal/otelschema/_testdata/model/version.properties new file mode 100644 index 00000000..03770cf2 --- /dev/null +++ b/internal/otelschema/_testdata/model/version.properties @@ -0,0 +1 @@ +version=1 diff --git a/internal/otelschema/generate.go b/internal/otelschema/generate.go new file mode 100644 index 00000000..94dbd3f0 --- /dev/null +++ b/internal/otelschema/generate.go @@ -0,0 +1,3 @@ +package otelschema + +//go:generate go run github.com/ogen-go/ogen/cmd/jschemagen --target group.go --package otelschema schema.yml diff --git a/internal/otelschema/group.go b/internal/otelschema/group.go new file mode 100644 index 00000000..d7aac38a --- /dev/null +++ b/internal/otelschema/group.go @@ -0,0 +1,2389 @@ +// Code generated by ogen, DO NOT EDIT. + +package otelschema + +import ( + "fmt" + "math/bits" + "strconv" + + "github.com/go-faster/errors" + "github.com/go-faster/jx" + + "github.com/ogen-go/ogen/validate" +) + +// Ref: #/$defs/attribute +type Attribute struct { + ID string `json:"id"` + SamplingRelevant OptBool `json:"sampling_relevant"` + Type AttributeType `json:"type"` + Brief OptString `json:"brief"` + Note OptString `json:"note"` + Tag OptString `json:"tag"` + Examples jx.Raw `json:"examples"` + RequirementLevel OptRequirementLevel `json:"requirement_level"` + Stability OptString `json:"stability"` +} + +// GetID returns the value of ID. +func (s *Attribute) GetID() string { + return s.ID +} + +// GetSamplingRelevant returns the value of SamplingRelevant. +func (s *Attribute) GetSamplingRelevant() OptBool { + return s.SamplingRelevant +} + +// GetType returns the value of Type. +func (s *Attribute) GetType() AttributeType { + return s.Type +} + +// GetBrief returns the value of Brief. +func (s *Attribute) GetBrief() OptString { + return s.Brief +} + +// GetNote returns the value of Note. +func (s *Attribute) GetNote() OptString { + return s.Note +} + +// GetTag returns the value of Tag. +func (s *Attribute) GetTag() OptString { + return s.Tag +} + +// GetExamples returns the value of Examples. +func (s *Attribute) GetExamples() jx.Raw { + return s.Examples +} + +// GetRequirementLevel returns the value of RequirementLevel. +func (s *Attribute) GetRequirementLevel() OptRequirementLevel { + return s.RequirementLevel +} + +// GetStability returns the value of Stability. +func (s *Attribute) GetStability() OptString { + return s.Stability +} + +// SetID sets the value of ID. +func (s *Attribute) SetID(val string) { + s.ID = val +} + +// SetSamplingRelevant sets the value of SamplingRelevant. +func (s *Attribute) SetSamplingRelevant(val OptBool) { + s.SamplingRelevant = val +} + +// SetType sets the value of Type. +func (s *Attribute) SetType(val AttributeType) { + s.Type = val +} + +// SetBrief sets the value of Brief. +func (s *Attribute) SetBrief(val OptString) { + s.Brief = val +} + +// SetNote sets the value of Note. +func (s *Attribute) SetNote(val OptString) { + s.Note = val +} + +// SetTag sets the value of Tag. +func (s *Attribute) SetTag(val OptString) { + s.Tag = val +} + +// SetExamples sets the value of Examples. +func (s *Attribute) SetExamples(val jx.Raw) { + s.Examples = val +} + +// SetRequirementLevel sets the value of RequirementLevel. +func (s *Attribute) SetRequirementLevel(val OptRequirementLevel) { + s.RequirementLevel = val +} + +// SetStability sets the value of Stability. +func (s *Attribute) SetStability(val OptString) { + s.Stability = val +} + +// Ref: #/$defs/attributeReference +type AttributeReference struct { + Ref string `json:"ref"` + Brief OptString `json:"brief"` + Note OptString `json:"note"` + Tag OptString `json:"tag"` + Examples []jx.Raw `json:"examples"` + RequirementLevel OptRequirementLevel `json:"requirement_level"` + SamplingRelevant OptBool `json:"sampling_relevant"` +} + +// GetRef returns the value of Ref. +func (s *AttributeReference) GetRef() string { + return s.Ref +} + +// GetBrief returns the value of Brief. +func (s *AttributeReference) GetBrief() OptString { + return s.Brief +} + +// GetNote returns the value of Note. +func (s *AttributeReference) GetNote() OptString { + return s.Note +} + +// GetTag returns the value of Tag. +func (s *AttributeReference) GetTag() OptString { + return s.Tag +} + +// GetExamples returns the value of Examples. +func (s *AttributeReference) GetExamples() []jx.Raw { + return s.Examples +} + +// GetRequirementLevel returns the value of RequirementLevel. +func (s *AttributeReference) GetRequirementLevel() OptRequirementLevel { + return s.RequirementLevel +} + +// GetSamplingRelevant returns the value of SamplingRelevant. +func (s *AttributeReference) GetSamplingRelevant() OptBool { + return s.SamplingRelevant +} + +// SetRef sets the value of Ref. +func (s *AttributeReference) SetRef(val string) { + s.Ref = val +} + +// SetBrief sets the value of Brief. +func (s *AttributeReference) SetBrief(val OptString) { + s.Brief = val +} + +// SetNote sets the value of Note. +func (s *AttributeReference) SetNote(val OptString) { + s.Note = val +} + +// SetTag sets the value of Tag. +func (s *AttributeReference) SetTag(val OptString) { + s.Tag = val +} + +// SetExamples sets the value of Examples. +func (s *AttributeReference) SetExamples(val []jx.Raw) { + s.Examples = val +} + +// SetRequirementLevel sets the value of RequirementLevel. +func (s *AttributeReference) SetRequirementLevel(val OptRequirementLevel) { + s.RequirementLevel = val +} + +// SetSamplingRelevant sets the value of SamplingRelevant. +func (s *AttributeReference) SetSamplingRelevant(val OptBool) { + s.SamplingRelevant = val +} + +// AttributeType represents sum type. +type AttributeType struct { + Type AttributeTypeType // switch on this field + Enum Enum + String string +} + +// AttributeTypeType is oneOf type of AttributeType. +type AttributeTypeType string + +// Possible values for AttributeTypeType. +const ( + EnumAttributeType AttributeTypeType = "Enum" + StringAttributeType AttributeTypeType = "string" +) + +// IsEnum reports whether AttributeType is Enum. +func (s AttributeType) IsEnum() bool { return s.Type == EnumAttributeType } + +// IsString reports whether AttributeType is string. +func (s AttributeType) IsString() bool { return s.Type == StringAttributeType } + +// SetEnum sets AttributeType to Enum. +func (s *AttributeType) SetEnum(v Enum) { + s.Type = EnumAttributeType + s.Enum = v +} + +// GetEnum returns Enum and true boolean if AttributeType is Enum. +func (s AttributeType) GetEnum() (v Enum, ok bool) { + if !s.IsEnum() { + return v, false + } + return s.Enum, true +} + +// NewEnumAttributeType returns new AttributeType from Enum. +func NewEnumAttributeType(v Enum) AttributeType { + var s AttributeType + s.SetEnum(v) + return s +} + +// SetString sets AttributeType to string. +func (s *AttributeType) SetString(v string) { + s.Type = StringAttributeType + s.String = v +} + +// GetString returns string and true boolean if AttributeType is string. +func (s AttributeType) GetString() (v string, ok bool) { + if !s.IsString() { + return v, false + } + return s.String, true +} + +// NewStringAttributeType returns new AttributeType from string. +func NewStringAttributeType(v string) AttributeType { + var s AttributeType + s.SetString(v) + return s +} + +// Ref: #/$defs/enum +type Enum struct { + AllowCustomValues OptBool `json:"allow_custom_values"` + Members []EnumMembersItem `json:"members"` +} + +// GetAllowCustomValues returns the value of AllowCustomValues. +func (s *Enum) GetAllowCustomValues() OptBool { + return s.AllowCustomValues +} + +// GetMembers returns the value of Members. +func (s *Enum) GetMembers() []EnumMembersItem { + return s.Members +} + +// SetAllowCustomValues sets the value of AllowCustomValues. +func (s *Enum) SetAllowCustomValues(val OptBool) { + s.AllowCustomValues = val +} + +// SetMembers sets the value of Members. +func (s *Enum) SetMembers(val []EnumMembersItem) { + s.Members = val +} + +type EnumMembersItem struct { + ID string `json:"id"` + Note OptString `json:"note"` + Value EnumMembersItemValue `json:"value"` + Brief OptString `json:"brief"` +} + +// GetID returns the value of ID. +func (s *EnumMembersItem) GetID() string { + return s.ID +} + +// GetNote returns the value of Note. +func (s *EnumMembersItem) GetNote() OptString { + return s.Note +} + +// GetValue returns the value of Value. +func (s *EnumMembersItem) GetValue() EnumMembersItemValue { + return s.Value +} + +// GetBrief returns the value of Brief. +func (s *EnumMembersItem) GetBrief() OptString { + return s.Brief +} + +// SetID sets the value of ID. +func (s *EnumMembersItem) SetID(val string) { + s.ID = val +} + +// SetNote sets the value of Note. +func (s *EnumMembersItem) SetNote(val OptString) { + s.Note = val +} + +// SetValue sets the value of Value. +func (s *EnumMembersItem) SetValue(val EnumMembersItemValue) { + s.Value = val +} + +// SetBrief sets the value of Brief. +func (s *EnumMembersItem) SetBrief(val OptString) { + s.Brief = val +} + +// EnumMembersItemValue represents sum type. +type EnumMembersItemValue struct { + Type EnumMembersItemValueType // switch on this field + String string + Float64 float64 +} + +// EnumMembersItemValueType is oneOf type of EnumMembersItemValue. +type EnumMembersItemValueType string + +// Possible values for EnumMembersItemValueType. +const ( + StringEnumMembersItemValue EnumMembersItemValueType = "string" + Float64EnumMembersItemValue EnumMembersItemValueType = "float64" +) + +// IsString reports whether EnumMembersItemValue is string. +func (s EnumMembersItemValue) IsString() bool { return s.Type == StringEnumMembersItemValue } + +// IsFloat64 reports whether EnumMembersItemValue is float64. +func (s EnumMembersItemValue) IsFloat64() bool { return s.Type == Float64EnumMembersItemValue } + +// SetString sets EnumMembersItemValue to string. +func (s *EnumMembersItemValue) SetString(v string) { + s.Type = StringEnumMembersItemValue + s.String = v +} + +// GetString returns string and true boolean if EnumMembersItemValue is string. +func (s EnumMembersItemValue) GetString() (v string, ok bool) { + if !s.IsString() { + return v, false + } + return s.String, true +} + +// NewStringEnumMembersItemValue returns new EnumMembersItemValue from string. +func NewStringEnumMembersItemValue(v string) EnumMembersItemValue { + var s EnumMembersItemValue + s.SetString(v) + return s +} + +// SetFloat64 sets EnumMembersItemValue to float64. +func (s *EnumMembersItemValue) SetFloat64(v float64) { + s.Type = Float64EnumMembersItemValue + s.Float64 = v +} + +// GetFloat64 returns float64 and true boolean if EnumMembersItemValue is float64. +func (s EnumMembersItemValue) GetFloat64() (v float64, ok bool) { + if !s.IsFloat64() { + return v, false + } + return s.Float64, true +} + +// NewFloat64EnumMembersItemValue returns new EnumMembersItemValue from float64. +func NewFloat64EnumMembersItemValue(v float64) EnumMembersItemValue { + var s EnumMembersItemValue + s.SetFloat64(v) + return s +} + +// NewOptBool returns new OptBool with value set to v. +func NewOptBool(v bool) OptBool { + return OptBool{ + Value: v, + Set: true, + } +} + +// OptBool is optional bool. +type OptBool struct { + Value bool + Set bool +} + +// IsSet returns true if OptBool was set. +func (o OptBool) IsSet() bool { return o.Set } + +// Reset unsets value. +func (o *OptBool) Reset() { + var v bool + o.Value = v + o.Set = false +} + +// SetTo sets value to v. +func (o *OptBool) SetTo(v bool) { + o.Set = true + o.Value = v +} + +// Get returns value and boolean that denotes whether value was set. +func (o OptBool) Get() (v bool, ok bool) { + if !o.Set { + return v, false + } + return o.Value, true +} + +// Or returns value if set, or given parameter if does not. +func (o OptBool) Or(d bool) bool { + if v, ok := o.Get(); ok { + return v + } + return d +} + +// NewOptRequirementLevel returns new OptRequirementLevel with value set to v. +func NewOptRequirementLevel(v RequirementLevel) OptRequirementLevel { + return OptRequirementLevel{ + Value: v, + Set: true, + } +} + +// OptRequirementLevel is optional RequirementLevel. +type OptRequirementLevel struct { + Value RequirementLevel + Set bool +} + +// IsSet returns true if OptRequirementLevel was set. +func (o OptRequirementLevel) IsSet() bool { return o.Set } + +// Reset unsets value. +func (o *OptRequirementLevel) Reset() { + var v RequirementLevel + o.Value = v + o.Set = false +} + +// SetTo sets value to v. +func (o *OptRequirementLevel) SetTo(v RequirementLevel) { + o.Set = true + o.Value = v +} + +// Get returns value and boolean that denotes whether value was set. +func (o OptRequirementLevel) Get() (v RequirementLevel, ok bool) { + if !o.Set { + return v, false + } + return o.Value, true +} + +// Or returns value if set, or given parameter if does not. +func (o OptRequirementLevel) Or(d RequirementLevel) RequirementLevel { + if v, ok := o.Get(); ok { + return v + } + return d +} + +// NewOptString returns new OptString with value set to v. +func NewOptString(v string) OptString { + return OptString{ + Value: v, + Set: true, + } +} + +// OptString is optional string. +type OptString struct { + Value string + Set bool +} + +// IsSet returns true if OptString was set. +func (o OptString) IsSet() bool { return o.Set } + +// Reset unsets value. +func (o *OptString) Reset() { + var v string + o.Value = v + o.Set = false +} + +// SetTo sets value to v. +func (o *OptString) SetTo(v string) { + o.Set = true + o.Value = v +} + +// Get returns value and boolean that denotes whether value was set. +func (o OptString) Get() (v string, ok bool) { + if !o.Set { + return v, false + } + return o.Value, true +} + +// Or returns value if set, or given parameter if does not. +func (o OptString) Or(d string) string { + if v, ok := o.Get(); ok { + return v + } + return d +} + +// Ref: #/$defs/requirementLevel +// RequirementLevel represents sum type. +type RequirementLevel struct { + Type RequirementLevelType // switch on this field + String string + RequirementLevel1 RequirementLevel1 +} + +// RequirementLevelType is oneOf type of RequirementLevel. +type RequirementLevelType string + +// Possible values for RequirementLevelType. +const ( + StringRequirementLevel RequirementLevelType = "string" + RequirementLevel1RequirementLevel RequirementLevelType = "RequirementLevel1" +) + +// IsString reports whether RequirementLevel is string. +func (s RequirementLevel) IsString() bool { return s.Type == StringRequirementLevel } + +// IsRequirementLevel1 reports whether RequirementLevel is RequirementLevel1. +func (s RequirementLevel) IsRequirementLevel1() bool { + return s.Type == RequirementLevel1RequirementLevel +} + +// SetString sets RequirementLevel to string. +func (s *RequirementLevel) SetString(v string) { + s.Type = StringRequirementLevel + s.String = v +} + +// GetString returns string and true boolean if RequirementLevel is string. +func (s RequirementLevel) GetString() (v string, ok bool) { + if !s.IsString() { + return v, false + } + return s.String, true +} + +// NewStringRequirementLevel returns new RequirementLevel from string. +func NewStringRequirementLevel(v string) RequirementLevel { + var s RequirementLevel + s.SetString(v) + return s +} + +// SetRequirementLevel1 sets RequirementLevel to RequirementLevel1. +func (s *RequirementLevel) SetRequirementLevel1(v RequirementLevel1) { + s.Type = RequirementLevel1RequirementLevel + s.RequirementLevel1 = v +} + +// GetRequirementLevel1 returns RequirementLevel1 and true boolean if RequirementLevel is RequirementLevel1. +func (s RequirementLevel) GetRequirementLevel1() (v RequirementLevel1, ok bool) { + if !s.IsRequirementLevel1() { + return v, false + } + return s.RequirementLevel1, true +} + +// NewRequirementLevel1RequirementLevel returns new RequirementLevel from RequirementLevel1. +func NewRequirementLevel1RequirementLevel(v RequirementLevel1) RequirementLevel { + var s RequirementLevel + s.SetRequirementLevel1(v) + return s +} + +type RequirementLevel1 map[string]string + +func (s *RequirementLevel1) init() RequirementLevel1 { + m := *s + if m == nil { + m = map[string]string{} + *s = m + } + return m +} + +type Type struct { + Groups []TypeGroupsItem `json:"groups"` +} + +// GetGroups returns the value of Groups. +func (s *Type) GetGroups() []TypeGroupsItem { + return s.Groups +} + +// SetGroups sets the value of Groups. +func (s *Type) SetGroups(val []TypeGroupsItem) { + s.Groups = val +} + +type TypeGroupsItem struct { + ID string `json:"id"` + Prefix OptString `json:"prefix"` + Type OptString `json:"type"` + Brief OptString `json:"brief"` + Note OptString `json:"note"` + Attributes []TypeGroupsItemAttributesItem `json:"attributes"` +} + +// GetID returns the value of ID. +func (s *TypeGroupsItem) GetID() string { + return s.ID +} + +// GetPrefix returns the value of Prefix. +func (s *TypeGroupsItem) GetPrefix() OptString { + return s.Prefix +} + +// GetType returns the value of Type. +func (s *TypeGroupsItem) GetType() OptString { + return s.Type +} + +// GetBrief returns the value of Brief. +func (s *TypeGroupsItem) GetBrief() OptString { + return s.Brief +} + +// GetNote returns the value of Note. +func (s *TypeGroupsItem) GetNote() OptString { + return s.Note +} + +// GetAttributes returns the value of Attributes. +func (s *TypeGroupsItem) GetAttributes() []TypeGroupsItemAttributesItem { + return s.Attributes +} + +// SetID sets the value of ID. +func (s *TypeGroupsItem) SetID(val string) { + s.ID = val +} + +// SetPrefix sets the value of Prefix. +func (s *TypeGroupsItem) SetPrefix(val OptString) { + s.Prefix = val +} + +// SetType sets the value of Type. +func (s *TypeGroupsItem) SetType(val OptString) { + s.Type = val +} + +// SetBrief sets the value of Brief. +func (s *TypeGroupsItem) SetBrief(val OptString) { + s.Brief = val +} + +// SetNote sets the value of Note. +func (s *TypeGroupsItem) SetNote(val OptString) { + s.Note = val +} + +// SetAttributes sets the value of Attributes. +func (s *TypeGroupsItem) SetAttributes(val []TypeGroupsItemAttributesItem) { + s.Attributes = val +} + +// TypeGroupsItemAttributesItem represents sum type. +type TypeGroupsItemAttributesItem struct { + Type TypeGroupsItemAttributesItemType // switch on this field + AttributeReference AttributeReference + Attribute Attribute +} + +// TypeGroupsItemAttributesItemType is oneOf type of TypeGroupsItemAttributesItem. +type TypeGroupsItemAttributesItemType string + +// Possible values for TypeGroupsItemAttributesItemType. +const ( + AttributeReferenceTypeGroupsItemAttributesItem TypeGroupsItemAttributesItemType = "AttributeReference" + AttributeTypeGroupsItemAttributesItem TypeGroupsItemAttributesItemType = "Attribute" +) + +// IsAttributeReference reports whether TypeGroupsItemAttributesItem is AttributeReference. +func (s TypeGroupsItemAttributesItem) IsAttributeReference() bool { + return s.Type == AttributeReferenceTypeGroupsItemAttributesItem +} + +// IsAttribute reports whether TypeGroupsItemAttributesItem is Attribute. +func (s TypeGroupsItemAttributesItem) IsAttribute() bool { + return s.Type == AttributeTypeGroupsItemAttributesItem +} + +// SetAttributeReference sets TypeGroupsItemAttributesItem to AttributeReference. +func (s *TypeGroupsItemAttributesItem) SetAttributeReference(v AttributeReference) { + s.Type = AttributeReferenceTypeGroupsItemAttributesItem + s.AttributeReference = v +} + +// GetAttributeReference returns AttributeReference and true boolean if TypeGroupsItemAttributesItem is AttributeReference. +func (s TypeGroupsItemAttributesItem) GetAttributeReference() (v AttributeReference, ok bool) { + if !s.IsAttributeReference() { + return v, false + } + return s.AttributeReference, true +} + +// NewAttributeReferenceTypeGroupsItemAttributesItem returns new TypeGroupsItemAttributesItem from AttributeReference. +func NewAttributeReferenceTypeGroupsItemAttributesItem(v AttributeReference) TypeGroupsItemAttributesItem { + var s TypeGroupsItemAttributesItem + s.SetAttributeReference(v) + return s +} + +// SetAttribute sets TypeGroupsItemAttributesItem to Attribute. +func (s *TypeGroupsItemAttributesItem) SetAttribute(v Attribute) { + s.Type = AttributeTypeGroupsItemAttributesItem + s.Attribute = v +} + +// GetAttribute returns Attribute and true boolean if TypeGroupsItemAttributesItem is Attribute. +func (s TypeGroupsItemAttributesItem) GetAttribute() (v Attribute, ok bool) { + if !s.IsAttribute() { + return v, false + } + return s.Attribute, true +} + +// NewAttributeTypeGroupsItemAttributesItem returns new TypeGroupsItemAttributesItem from Attribute. +func NewAttributeTypeGroupsItemAttributesItem(v Attribute) TypeGroupsItemAttributesItem { + var s TypeGroupsItemAttributesItem + s.SetAttribute(v) + return s +} + +// Encode implements json.Marshaler. +func (s *Attribute) Encode(e *jx.Encoder) { + e.ObjStart() + s.encodeFields(e) + e.ObjEnd() +} + +// encodeFields encodes fields. +func (s *Attribute) encodeFields(e *jx.Encoder) { + { + e.FieldStart("id") + e.Str(s.ID) + } + { + if s.SamplingRelevant.Set { + e.FieldStart("sampling_relevant") + s.SamplingRelevant.Encode(e) + } + } + { + e.FieldStart("type") + s.Type.Encode(e) + } + { + if s.Brief.Set { + e.FieldStart("brief") + s.Brief.Encode(e) + } + } + { + if s.Note.Set { + e.FieldStart("note") + s.Note.Encode(e) + } + } + { + if s.Tag.Set { + e.FieldStart("tag") + s.Tag.Encode(e) + } + } + { + if len(s.Examples) != 0 { + e.FieldStart("examples") + e.Raw(s.Examples) + } + } + { + if s.RequirementLevel.Set { + e.FieldStart("requirement_level") + s.RequirementLevel.Encode(e) + } + } + { + if s.Stability.Set { + e.FieldStart("stability") + s.Stability.Encode(e) + } + } +} + +var jsonFieldsNameOfAttribute = [9]string{ + 0: "id", + 1: "sampling_relevant", + 2: "type", + 3: "brief", + 4: "note", + 5: "tag", + 6: "examples", + 7: "requirement_level", + 8: "stability", +} + +// Decode decodes Attribute from json. +func (s *Attribute) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode Attribute to nil") + } + var requiredBitSet [2]uint8 + + if err := d.ObjBytes(func(d *jx.Decoder, k []byte) error { + switch string(k) { + case "id": + requiredBitSet[0] |= 1 << 0 + if err := func() error { + v, err := d.Str() + s.ID = string(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"id\"") + } + case "sampling_relevant": + if err := func() error { + s.SamplingRelevant.Reset() + if err := s.SamplingRelevant.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"sampling_relevant\"") + } + case "type": + requiredBitSet[0] |= 1 << 2 + if err := func() error { + if err := s.Type.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"type\"") + } + case "brief": + if err := func() error { + s.Brief.Reset() + if err := s.Brief.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"brief\"") + } + case "note": + if err := func() error { + s.Note.Reset() + if err := s.Note.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"note\"") + } + case "tag": + if err := func() error { + s.Tag.Reset() + if err := s.Tag.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"tag\"") + } + case "examples": + if err := func() error { + v, err := d.RawAppend(nil) + s.Examples = jx.Raw(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"examples\"") + } + case "requirement_level": + if err := func() error { + s.RequirementLevel.Reset() + if err := s.RequirementLevel.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"requirement_level\"") + } + case "stability": + if err := func() error { + s.Stability.Reset() + if err := s.Stability.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"stability\"") + } + default: + return errors.Errorf("unexpected field %q", k) + } + return nil + }); err != nil { + return errors.Wrap(err, "decode Attribute") + } + // Validate required fields. + var failures []validate.FieldError + for i, mask := range [2]uint8{ + 0b00000101, + 0b00000000, + } { + if result := (requiredBitSet[i] & mask) ^ mask; result != 0 { + // Mask only required fields and check equality to mask using XOR. + // + // If XOR result is not zero, result is not equal to expected, so some fields are missed. + // Bits of fields which would be set are actually bits of missed fields. + missed := bits.OnesCount8(result) + for bitN := 0; bitN < missed; bitN++ { + bitIdx := bits.TrailingZeros8(result) + fieldIdx := i*8 + bitIdx + var name string + if fieldIdx < len(jsonFieldsNameOfAttribute) { + name = jsonFieldsNameOfAttribute[fieldIdx] + } else { + name = strconv.Itoa(fieldIdx) + } + failures = append(failures, validate.FieldError{ + Name: name, + Error: validate.ErrFieldRequired, + }) + // Reset bit. + result &^= 1 << bitIdx + } + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s *Attribute) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *Attribute) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode implements json.Marshaler. +func (s *AttributeReference) Encode(e *jx.Encoder) { + e.ObjStart() + s.encodeFields(e) + e.ObjEnd() +} + +// encodeFields encodes fields. +func (s *AttributeReference) encodeFields(e *jx.Encoder) { + { + e.FieldStart("ref") + e.Str(s.Ref) + } + { + if s.Brief.Set { + e.FieldStart("brief") + s.Brief.Encode(e) + } + } + { + if s.Note.Set { + e.FieldStart("note") + s.Note.Encode(e) + } + } + { + if s.Tag.Set { + e.FieldStart("tag") + s.Tag.Encode(e) + } + } + { + if s.Examples != nil { + e.FieldStart("examples") + e.ArrStart() + for _, elem := range s.Examples { + if len(elem) != 0 { + e.Raw(elem) + } + } + e.ArrEnd() + } + } + { + if s.RequirementLevel.Set { + e.FieldStart("requirement_level") + s.RequirementLevel.Encode(e) + } + } + { + if s.SamplingRelevant.Set { + e.FieldStart("sampling_relevant") + s.SamplingRelevant.Encode(e) + } + } +} + +var jsonFieldsNameOfAttributeReference = [7]string{ + 0: "ref", + 1: "brief", + 2: "note", + 3: "tag", + 4: "examples", + 5: "requirement_level", + 6: "sampling_relevant", +} + +// Decode decodes AttributeReference from json. +func (s *AttributeReference) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode AttributeReference to nil") + } + var requiredBitSet [1]uint8 + + if err := d.ObjBytes(func(d *jx.Decoder, k []byte) error { + switch string(k) { + case "ref": + requiredBitSet[0] |= 1 << 0 + if err := func() error { + v, err := d.Str() + s.Ref = string(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"ref\"") + } + case "brief": + if err := func() error { + s.Brief.Reset() + if err := s.Brief.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"brief\"") + } + case "note": + if err := func() error { + s.Note.Reset() + if err := s.Note.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"note\"") + } + case "tag": + if err := func() error { + s.Tag.Reset() + if err := s.Tag.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"tag\"") + } + case "examples": + if err := func() error { + s.Examples = make([]jx.Raw, 0) + if err := d.Arr(func(d *jx.Decoder) error { + var elem jx.Raw + v, err := d.RawAppend(nil) + elem = jx.Raw(v) + if err != nil { + return err + } + s.Examples = append(s.Examples, elem) + return nil + }); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"examples\"") + } + case "requirement_level": + if err := func() error { + s.RequirementLevel.Reset() + if err := s.RequirementLevel.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"requirement_level\"") + } + case "sampling_relevant": + if err := func() error { + s.SamplingRelevant.Reset() + if err := s.SamplingRelevant.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"sampling_relevant\"") + } + default: + return errors.Errorf("unexpected field %q", k) + } + return nil + }); err != nil { + return errors.Wrap(err, "decode AttributeReference") + } + // Validate required fields. + var failures []validate.FieldError + for i, mask := range [1]uint8{ + 0b00000001, + } { + if result := (requiredBitSet[i] & mask) ^ mask; result != 0 { + // Mask only required fields and check equality to mask using XOR. + // + // If XOR result is not zero, result is not equal to expected, so some fields are missed. + // Bits of fields which would be set are actually bits of missed fields. + missed := bits.OnesCount8(result) + for bitN := 0; bitN < missed; bitN++ { + bitIdx := bits.TrailingZeros8(result) + fieldIdx := i*8 + bitIdx + var name string + if fieldIdx < len(jsonFieldsNameOfAttributeReference) { + name = jsonFieldsNameOfAttributeReference[fieldIdx] + } else { + name = strconv.Itoa(fieldIdx) + } + failures = append(failures, validate.FieldError{ + Name: name, + Error: validate.ErrFieldRequired, + }) + // Reset bit. + result &^= 1 << bitIdx + } + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s *AttributeReference) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *AttributeReference) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode encodes AttributeType as json. +func (s AttributeType) Encode(e *jx.Encoder) { + switch s.Type { + case EnumAttributeType: + s.Enum.Encode(e) + case StringAttributeType: + e.Str(s.String) + } +} + +// Decode decodes AttributeType from json. +func (s *AttributeType) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode AttributeType to nil") + } + // Sum type type_discriminator. + switch t := d.Next(); t { + case jx.Object: + if err := s.Enum.Decode(d); err != nil { + return err + } + s.Type = EnumAttributeType + case jx.String: + v, err := d.Str() + s.String = string(v) + if err != nil { + return err + } + s.Type = StringAttributeType + default: + return errors.Errorf("unexpected json type %q", t) + } + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s AttributeType) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *AttributeType) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode implements json.Marshaler. +func (s *Enum) Encode(e *jx.Encoder) { + e.ObjStart() + s.encodeFields(e) + e.ObjEnd() +} + +// encodeFields encodes fields. +func (s *Enum) encodeFields(e *jx.Encoder) { + { + if s.AllowCustomValues.Set { + e.FieldStart("allow_custom_values") + s.AllowCustomValues.Encode(e) + } + } + { + e.FieldStart("members") + e.ArrStart() + for _, elem := range s.Members { + elem.Encode(e) + } + e.ArrEnd() + } +} + +var jsonFieldsNameOfEnum = [2]string{ + 0: "allow_custom_values", + 1: "members", +} + +// Decode decodes Enum from json. +func (s *Enum) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode Enum to nil") + } + var requiredBitSet [1]uint8 + + if err := d.ObjBytes(func(d *jx.Decoder, k []byte) error { + switch string(k) { + case "allow_custom_values": + if err := func() error { + s.AllowCustomValues.Reset() + if err := s.AllowCustomValues.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"allow_custom_values\"") + } + case "members": + requiredBitSet[0] |= 1 << 1 + if err := func() error { + s.Members = make([]EnumMembersItem, 0) + if err := d.Arr(func(d *jx.Decoder) error { + var elem EnumMembersItem + if err := elem.Decode(d); err != nil { + return err + } + s.Members = append(s.Members, elem) + return nil + }); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"members\"") + } + default: + return errors.Errorf("unexpected field %q", k) + } + return nil + }); err != nil { + return errors.Wrap(err, "decode Enum") + } + // Validate required fields. + var failures []validate.FieldError + for i, mask := range [1]uint8{ + 0b00000010, + } { + if result := (requiredBitSet[i] & mask) ^ mask; result != 0 { + // Mask only required fields and check equality to mask using XOR. + // + // If XOR result is not zero, result is not equal to expected, so some fields are missed. + // Bits of fields which would be set are actually bits of missed fields. + missed := bits.OnesCount8(result) + for bitN := 0; bitN < missed; bitN++ { + bitIdx := bits.TrailingZeros8(result) + fieldIdx := i*8 + bitIdx + var name string + if fieldIdx < len(jsonFieldsNameOfEnum) { + name = jsonFieldsNameOfEnum[fieldIdx] + } else { + name = strconv.Itoa(fieldIdx) + } + failures = append(failures, validate.FieldError{ + Name: name, + Error: validate.ErrFieldRequired, + }) + // Reset bit. + result &^= 1 << bitIdx + } + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s *Enum) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *Enum) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode implements json.Marshaler. +func (s *EnumMembersItem) Encode(e *jx.Encoder) { + e.ObjStart() + s.encodeFields(e) + e.ObjEnd() +} + +// encodeFields encodes fields. +func (s *EnumMembersItem) encodeFields(e *jx.Encoder) { + { + e.FieldStart("id") + e.Str(s.ID) + } + { + if s.Note.Set { + e.FieldStart("note") + s.Note.Encode(e) + } + } + { + e.FieldStart("value") + s.Value.Encode(e) + } + { + if s.Brief.Set { + e.FieldStart("brief") + s.Brief.Encode(e) + } + } +} + +var jsonFieldsNameOfEnumMembersItem = [4]string{ + 0: "id", + 1: "note", + 2: "value", + 3: "brief", +} + +// Decode decodes EnumMembersItem from json. +func (s *EnumMembersItem) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode EnumMembersItem to nil") + } + var requiredBitSet [1]uint8 + + if err := d.ObjBytes(func(d *jx.Decoder, k []byte) error { + switch string(k) { + case "id": + requiredBitSet[0] |= 1 << 0 + if err := func() error { + v, err := d.Str() + s.ID = string(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"id\"") + } + case "note": + if err := func() error { + s.Note.Reset() + if err := s.Note.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"note\"") + } + case "value": + requiredBitSet[0] |= 1 << 2 + if err := func() error { + if err := s.Value.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"value\"") + } + case "brief": + if err := func() error { + s.Brief.Reset() + if err := s.Brief.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"brief\"") + } + default: + return errors.Errorf("unexpected field %q", k) + } + return nil + }); err != nil { + return errors.Wrap(err, "decode EnumMembersItem") + } + // Validate required fields. + var failures []validate.FieldError + for i, mask := range [1]uint8{ + 0b00000101, + } { + if result := (requiredBitSet[i] & mask) ^ mask; result != 0 { + // Mask only required fields and check equality to mask using XOR. + // + // If XOR result is not zero, result is not equal to expected, so some fields are missed. + // Bits of fields which would be set are actually bits of missed fields. + missed := bits.OnesCount8(result) + for bitN := 0; bitN < missed; bitN++ { + bitIdx := bits.TrailingZeros8(result) + fieldIdx := i*8 + bitIdx + var name string + if fieldIdx < len(jsonFieldsNameOfEnumMembersItem) { + name = jsonFieldsNameOfEnumMembersItem[fieldIdx] + } else { + name = strconv.Itoa(fieldIdx) + } + failures = append(failures, validate.FieldError{ + Name: name, + Error: validate.ErrFieldRequired, + }) + // Reset bit. + result &^= 1 << bitIdx + } + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s *EnumMembersItem) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *EnumMembersItem) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode encodes EnumMembersItemValue as json. +func (s EnumMembersItemValue) Encode(e *jx.Encoder) { + switch s.Type { + case StringEnumMembersItemValue: + e.Str(s.String) + case Float64EnumMembersItemValue: + e.Float64(s.Float64) + } +} + +// Decode decodes EnumMembersItemValue from json. +func (s *EnumMembersItemValue) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode EnumMembersItemValue to nil") + } + // Sum type type_discriminator. + switch t := d.Next(); t { + case jx.Number: + v, err := d.Float64() + s.Float64 = float64(v) + if err != nil { + return err + } + s.Type = Float64EnumMembersItemValue + case jx.String: + v, err := d.Str() + s.String = string(v) + if err != nil { + return err + } + s.Type = StringEnumMembersItemValue + default: + return errors.Errorf("unexpected json type %q", t) + } + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s EnumMembersItemValue) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *EnumMembersItemValue) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode encodes bool as json. +func (o OptBool) Encode(e *jx.Encoder) { + if !o.Set { + return + } + e.Bool(bool(o.Value)) +} + +// Decode decodes bool from json. +func (o *OptBool) Decode(d *jx.Decoder) error { + if o == nil { + return errors.New("invalid: unable to decode OptBool to nil") + } + o.Set = true + v, err := d.Bool() + if err != nil { + return err + } + o.Value = bool(v) + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s OptBool) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *OptBool) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode encodes RequirementLevel as json. +func (o OptRequirementLevel) Encode(e *jx.Encoder) { + if !o.Set { + return + } + o.Value.Encode(e) +} + +// Decode decodes RequirementLevel from json. +func (o *OptRequirementLevel) Decode(d *jx.Decoder) error { + if o == nil { + return errors.New("invalid: unable to decode OptRequirementLevel to nil") + } + o.Set = true + if err := o.Value.Decode(d); err != nil { + return err + } + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s OptRequirementLevel) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *OptRequirementLevel) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode encodes string as json. +func (o OptString) Encode(e *jx.Encoder) { + if !o.Set { + return + } + e.Str(string(o.Value)) +} + +// Decode decodes string from json. +func (o *OptString) Decode(d *jx.Decoder) error { + if o == nil { + return errors.New("invalid: unable to decode OptString to nil") + } + o.Set = true + v, err := d.Str() + if err != nil { + return err + } + o.Value = string(v) + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s OptString) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *OptString) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode encodes RequirementLevel as json. +func (s RequirementLevel) Encode(e *jx.Encoder) { + switch s.Type { + case StringRequirementLevel: + e.Str(s.String) + case RequirementLevel1RequirementLevel: + s.RequirementLevel1.Encode(e) + } +} + +// Decode decodes RequirementLevel from json. +func (s *RequirementLevel) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode RequirementLevel to nil") + } + // Sum type type_discriminator. + switch t := d.Next(); t { + case jx.Object: + if err := s.RequirementLevel1.Decode(d); err != nil { + return err + } + s.Type = RequirementLevel1RequirementLevel + case jx.String: + v, err := d.Str() + s.String = string(v) + if err != nil { + return err + } + s.Type = StringRequirementLevel + default: + return errors.Errorf("unexpected json type %q", t) + } + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s RequirementLevel) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *RequirementLevel) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode implements json.Marshaler. +func (s RequirementLevel1) Encode(e *jx.Encoder) { + e.ObjStart() + s.encodeFields(e) + e.ObjEnd() +} + +// encodeFields implements json.Marshaler. +func (s RequirementLevel1) encodeFields(e *jx.Encoder) { + for k, elem := range s { + e.FieldStart(k) + + e.Str(elem) + } +} + +// Decode decodes RequirementLevel1 from json. +func (s *RequirementLevel1) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode RequirementLevel1 to nil") + } + m := s.init() + if err := d.ObjBytes(func(d *jx.Decoder, k []byte) error { + var elem string + if err := func() error { + v, err := d.Str() + elem = string(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrapf(err, "decode field %q", k) + } + m[string(k)] = elem + return nil + }); err != nil { + return errors.Wrap(err, "decode RequirementLevel1") + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s RequirementLevel1) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *RequirementLevel1) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode implements json.Marshaler. +func (s *Type) Encode(e *jx.Encoder) { + e.ObjStart() + s.encodeFields(e) + e.ObjEnd() +} + +// encodeFields encodes fields. +func (s *Type) encodeFields(e *jx.Encoder) { + { + e.FieldStart("groups") + e.ArrStart() + for _, elem := range s.Groups { + elem.Encode(e) + } + e.ArrEnd() + } +} + +var jsonFieldsNameOfType = [1]string{ + 0: "groups", +} + +// Decode decodes Type from json. +func (s *Type) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode Type to nil") + } + var requiredBitSet [1]uint8 + + if err := d.ObjBytes(func(d *jx.Decoder, k []byte) error { + switch string(k) { + case "groups": + requiredBitSet[0] |= 1 << 0 + if err := func() error { + s.Groups = make([]TypeGroupsItem, 0) + if err := d.Arr(func(d *jx.Decoder) error { + var elem TypeGroupsItem + if err := elem.Decode(d); err != nil { + return err + } + s.Groups = append(s.Groups, elem) + return nil + }); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"groups\"") + } + default: + return errors.Errorf("unexpected field %q", k) + } + return nil + }); err != nil { + return errors.Wrap(err, "decode Type") + } + // Validate required fields. + var failures []validate.FieldError + for i, mask := range [1]uint8{ + 0b00000001, + } { + if result := (requiredBitSet[i] & mask) ^ mask; result != 0 { + // Mask only required fields and check equality to mask using XOR. + // + // If XOR result is not zero, result is not equal to expected, so some fields are missed. + // Bits of fields which would be set are actually bits of missed fields. + missed := bits.OnesCount8(result) + for bitN := 0; bitN < missed; bitN++ { + bitIdx := bits.TrailingZeros8(result) + fieldIdx := i*8 + bitIdx + var name string + if fieldIdx < len(jsonFieldsNameOfType) { + name = jsonFieldsNameOfType[fieldIdx] + } else { + name = strconv.Itoa(fieldIdx) + } + failures = append(failures, validate.FieldError{ + Name: name, + Error: validate.ErrFieldRequired, + }) + // Reset bit. + result &^= 1 << bitIdx + } + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s *Type) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *Type) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode implements json.Marshaler. +func (s *TypeGroupsItem) Encode(e *jx.Encoder) { + e.ObjStart() + s.encodeFields(e) + e.ObjEnd() +} + +// encodeFields encodes fields. +func (s *TypeGroupsItem) encodeFields(e *jx.Encoder) { + { + e.FieldStart("id") + e.Str(s.ID) + } + { + if s.Prefix.Set { + e.FieldStart("prefix") + s.Prefix.Encode(e) + } + } + { + if s.Type.Set { + e.FieldStart("type") + s.Type.Encode(e) + } + } + { + if s.Brief.Set { + e.FieldStart("brief") + s.Brief.Encode(e) + } + } + { + if s.Note.Set { + e.FieldStart("note") + s.Note.Encode(e) + } + } + { + if s.Attributes != nil { + e.FieldStart("attributes") + e.ArrStart() + for _, elem := range s.Attributes { + elem.Encode(e) + } + e.ArrEnd() + } + } +} + +var jsonFieldsNameOfTypeGroupsItem = [6]string{ + 0: "id", + 1: "prefix", + 2: "type", + 3: "brief", + 4: "note", + 5: "attributes", +} + +// Decode decodes TypeGroupsItem from json. +func (s *TypeGroupsItem) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode TypeGroupsItem to nil") + } + var requiredBitSet [1]uint8 + + if err := d.ObjBytes(func(d *jx.Decoder, k []byte) error { + switch string(k) { + case "id": + requiredBitSet[0] |= 1 << 0 + if err := func() error { + v, err := d.Str() + s.ID = string(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"id\"") + } + case "prefix": + if err := func() error { + s.Prefix.Reset() + if err := s.Prefix.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"prefix\"") + } + case "type": + if err := func() error { + s.Type.Reset() + if err := s.Type.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"type\"") + } + case "brief": + if err := func() error { + s.Brief.Reset() + if err := s.Brief.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"brief\"") + } + case "note": + if err := func() error { + s.Note.Reset() + if err := s.Note.Decode(d); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"note\"") + } + case "attributes": + if err := func() error { + s.Attributes = make([]TypeGroupsItemAttributesItem, 0) + if err := d.Arr(func(d *jx.Decoder) error { + var elem TypeGroupsItemAttributesItem + if err := elem.Decode(d); err != nil { + return err + } + s.Attributes = append(s.Attributes, elem) + return nil + }); err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"attributes\"") + } + default: + return d.Skip() + } + return nil + }); err != nil { + return errors.Wrap(err, "decode TypeGroupsItem") + } + // Validate required fields. + var failures []validate.FieldError + for i, mask := range [1]uint8{ + 0b00000001, + } { + if result := (requiredBitSet[i] & mask) ^ mask; result != 0 { + // Mask only required fields and check equality to mask using XOR. + // + // If XOR result is not zero, result is not equal to expected, so some fields are missed. + // Bits of fields which would be set are actually bits of missed fields. + missed := bits.OnesCount8(result) + for bitN := 0; bitN < missed; bitN++ { + bitIdx := bits.TrailingZeros8(result) + fieldIdx := i*8 + bitIdx + var name string + if fieldIdx < len(jsonFieldsNameOfTypeGroupsItem) { + name = jsonFieldsNameOfTypeGroupsItem[fieldIdx] + } else { + name = strconv.Itoa(fieldIdx) + } + failures = append(failures, validate.FieldError{ + Name: name, + Error: validate.ErrFieldRequired, + }) + // Reset bit. + result &^= 1 << bitIdx + } + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s *TypeGroupsItem) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *TypeGroupsItem) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode encodes TypeGroupsItemAttributesItem as json. +func (s TypeGroupsItemAttributesItem) Encode(e *jx.Encoder) { + switch s.Type { + case AttributeReferenceTypeGroupsItemAttributesItem: + s.AttributeReference.Encode(e) + case AttributeTypeGroupsItemAttributesItem: + s.Attribute.Encode(e) + } +} + +func (s TypeGroupsItemAttributesItem) encodeFields(e *jx.Encoder) { + switch s.Type { + case AttributeReferenceTypeGroupsItemAttributesItem: + s.AttributeReference.encodeFields(e) + case AttributeTypeGroupsItemAttributesItem: + s.Attribute.encodeFields(e) + } +} + +// Decode decodes TypeGroupsItemAttributesItem from json. +func (s *TypeGroupsItemAttributesItem) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode TypeGroupsItemAttributesItem to nil") + } + // Sum type fields. + if typ := d.Next(); typ != jx.Object { + return errors.Errorf("unexpected json type %q", typ) + } + + var found bool + if err := d.Capture(func(d *jx.Decoder) error { + return d.ObjBytes(func(d *jx.Decoder, key []byte) error { + switch string(key) { + case "ref": + match := AttributeReferenceTypeGroupsItemAttributesItem + if found && s.Type != match { + s.Type = "" + return errors.Errorf("multiple oneOf matches: (%v, %v)", s.Type, match) + } + found = true + s.Type = match + case "id": + match := AttributeTypeGroupsItemAttributesItem + if found && s.Type != match { + s.Type = "" + return errors.Errorf("multiple oneOf matches: (%v, %v)", s.Type, match) + } + found = true + s.Type = match + case "type": + match := AttributeTypeGroupsItemAttributesItem + if found && s.Type != match { + s.Type = "" + return errors.Errorf("multiple oneOf matches: (%v, %v)", s.Type, match) + } + found = true + s.Type = match + case "stability": + match := AttributeTypeGroupsItemAttributesItem + if found && s.Type != match { + s.Type = "" + return errors.Errorf("multiple oneOf matches: (%v, %v)", s.Type, match) + } + found = true + s.Type = match + } + return d.Skip() + }) + }); err != nil { + return errors.Wrap(err, "capture") + } + if !found { + return errors.New("unable to detect sum type variant") + } + switch s.Type { + case AttributeReferenceTypeGroupsItemAttributesItem: + if err := s.AttributeReference.Decode(d); err != nil { + return err + } + case AttributeTypeGroupsItemAttributesItem: + if err := s.Attribute.Decode(d); err != nil { + return err + } + default: + return errors.Errorf("inferred invalid type: %s", s.Type) + } + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s TypeGroupsItemAttributesItem) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *TypeGroupsItemAttributesItem) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +func (s *Attribute) Validate() error { + if s == nil { + return validate.ErrNilPointer + } + + var failures []validate.FieldError + if err := func() error { + if err := s.Type.Validate(); err != nil { + return err + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: "type", + Error: err, + }) + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil +} + +func (s AttributeType) Validate() error { + switch s.Type { + case EnumAttributeType: + if err := s.Enum.Validate(); err != nil { + return err + } + return nil + case StringAttributeType: + return nil // no validation needed + default: + return errors.Errorf("invalid type %q", s.Type) + } +} + +func (s *Enum) Validate() error { + if s == nil { + return validate.ErrNilPointer + } + + var failures []validate.FieldError + if err := func() error { + if s.Members == nil { + return errors.New("nil is invalid value") + } + var failures []validate.FieldError + for i, elem := range s.Members { + if err := func() error { + if err := elem.Validate(); err != nil { + return err + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: fmt.Sprintf("[%d]", i), + Error: err, + }) + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: "members", + Error: err, + }) + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil +} + +func (s *EnumMembersItem) Validate() error { + if s == nil { + return validate.ErrNilPointer + } + + var failures []validate.FieldError + if err := func() error { + if err := s.Value.Validate(); err != nil { + return err + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: "value", + Error: err, + }) + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil +} + +func (s EnumMembersItemValue) Validate() error { + switch s.Type { + case StringEnumMembersItemValue: + return nil // no validation needed + case Float64EnumMembersItemValue: + if err := (validate.Float{}).Validate(float64(s.Float64)); err != nil { + return errors.Wrap(err, "float") + } + return nil + default: + return errors.Errorf("invalid type %q", s.Type) + } +} + +func (s *Type) Validate() error { + if s == nil { + return validate.ErrNilPointer + } + + var failures []validate.FieldError + if err := func() error { + if s.Groups == nil { + return errors.New("nil is invalid value") + } + var failures []validate.FieldError + for i, elem := range s.Groups { + if err := func() error { + if err := elem.Validate(); err != nil { + return err + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: fmt.Sprintf("[%d]", i), + Error: err, + }) + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: "groups", + Error: err, + }) + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil +} + +func (s *TypeGroupsItem) Validate() error { + if s == nil { + return validate.ErrNilPointer + } + + var failures []validate.FieldError + if err := func() error { + var failures []validate.FieldError + for i, elem := range s.Attributes { + if err := func() error { + if err := elem.Validate(); err != nil { + return err + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: fmt.Sprintf("[%d]", i), + Error: err, + }) + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: "attributes", + Error: err, + }) + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil +} + +func (s TypeGroupsItemAttributesItem) Validate() error { + switch s.Type { + case AttributeReferenceTypeGroupsItemAttributesItem: + return nil // no validation needed + case AttributeTypeGroupsItemAttributesItem: + if err := s.Attribute.Validate(); err != nil { + return err + } + return nil + default: + return errors.Errorf("invalid type %q", s.Type) + } +} diff --git a/internal/otelschema/group_test.go b/internal/otelschema/group_test.go new file mode 100644 index 00000000..98573e17 --- /dev/null +++ b/internal/otelschema/group_test.go @@ -0,0 +1,34 @@ +package otelschema + +import ( + "io/fs" + "os" + "path/filepath" + "testing" + + "github.com/stretchr/testify/require" + "sigs.k8s.io/yaml" +) + +func TestParse(t *testing.T) { + require.NoError(t, filepath.Walk(filepath.Join("_testdata", "model"), func(path string, info fs.FileInfo, err error) error { + require.NoError(t, err) + if info.IsDir() { + return nil + } + if filepath.Ext(path) != ".yaml" { + return nil + } + t.Run(path, func(t *testing.T) { + data, err := os.ReadFile(path) + require.NoError(t, err) + + var schema Type + jsonData, err := yaml.YAMLToJSON(data) + require.NoError(t, err) + t.Logf("json: %s", jsonData) + require.NoError(t, schema.UnmarshalJSON(jsonData)) + }) + return nil + })) +} diff --git a/internal/otelschema/otelschema.go b/internal/otelschema/otelschema.go new file mode 100644 index 00000000..106c52c4 --- /dev/null +++ b/internal/otelschema/otelschema.go @@ -0,0 +1,10 @@ +// Package otelschema implements reading OpenTelemetry semantic convention +// schemas. +// +// See: +// - [Model] +// - [Diff parser] +// +// [Model]: https://github.com/open-telemetry/semantic-conventions/tree/main/model +// [Diff parser]: https://github.com/open-telemetry/opentelemetry-go/tree/main/schema +package otelschema diff --git a/internal/otelschema/schema.yml b/internal/otelschema/schema.yml new file mode 100644 index 00000000..342c31d7 --- /dev/null +++ b/internal/otelschema/schema.yml @@ -0,0 +1,112 @@ +$schema: "http://json-schema.org/draft-07/schema#" + +$defs: + enum: + additionalProperties: false + type: object + required: + - members + properties: + allow_custom_values: + type: boolean + members: + type: array + items: + type: object + additionalProperties: false + properties: + id: + type: string + note: + type: string + value: + oneOf: + - type: string + - type: number + brief: + type: string + required: + - id + - value + requirementLevel: + oneOf: + - type: string + - type: object + additionalProperties: + type: string + attributeReference: + type: object + additionalProperties: false + required: + - ref + properties: + ref: + type: string + brief: + type: string + note: + type: string + tag: + type: string + examples: + type: array + items: {} + requirement_level: + $ref: "#/$defs/requirementLevel" + sampling_relevant: + type: boolean + attribute: + type: object + additionalProperties: false + properties: + id: + type: string + sampling_relevant: + type: boolean + type: + oneOf: + - $ref: "#/$defs/enum" + - type: string + brief: + type: string + note: + type: string + tag: + type: string + examples: {} + requirement_level: + $ref: "#/$defs/requirementLevel" + stability: + type: string + required: + - id + - type + +type: object +additionalProperties: false +properties: + groups: + type: array + items: + type: object + properties: + id: + type: string + prefix: + type: string + type: + type: string + brief: + type: string + note: + type: string + attributes: + type: array + items: + oneOf: + - $ref: "#/$defs/attributeReference" + - $ref: "#/$defs/attribute" + required: + - id +required: + - groups From 1c0b2f1e13c787630a133d8a6c12da616b627adf Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sat, 25 Nov 2023 14:36:47 +0300 Subject: [PATCH 049/112] feat(otelschema): add golden test --- .../otelschema/_golden/all_attributes.yaml | 716 ++++++++++++++++++ internal/otelschema/gold_test.go | 15 + internal/otelschema/group.go | 141 +++- internal/otelschema/list_test.go | 65 ++ internal/otelschema/schema.yml | 6 + 5 files changed, 934 insertions(+), 9 deletions(-) create mode 100644 internal/otelschema/_golden/all_attributes.yaml create mode 100644 internal/otelschema/gold_test.go create mode 100644 internal/otelschema/list_test.go diff --git a/internal/otelschema/_golden/all_attributes.yaml b/internal/otelschema/_golden/all_attributes.yaml new file mode 100644 index 00000000..82b9f002 --- /dev/null +++ b/internal/otelschema/_golden/all_attributes.yaml @@ -0,0 +1,716 @@ +- Name: exception.type + Type: string +- Name: exception.message + Type: string +- Name: exception.stacktrace + Type: string +- Name: faas.trigger + Type: enum +- Name: faas.invoked_name + Type: string +- Name: faas.invoked_provider + Type: enum +- Name: faas.invoked_region + Type: string +- Name: peer.service + Type: string +- Name: enduser.id + Type: string +- Name: enduser.role + Type: string +- Name: enduser.scope + Type: string +- Name: event.name + Type: string +- Name: log.record.uid + Type: string +- Name: log.iostream + Type: enum +- Name: log.file.name + Type: string +- Name: log.file.path + Type: string +- Name: log.file.name_resolved + Type: string +- Name: log.file.path_resolved + Type: string +- Name: ios.state + Type: enum +- Name: android.state + Type: enum +- Name: state + Type: enum +- Name: pool.name + Type: string +- Name: jvm.buffer.pool.name + Type: string +- Name: jvm.memory.type + Type: enum +- Name: jvm.memory.pool.name + Type: string +- Name: jvm.gc.name + Type: string +- Name: jvm.gc.action + Type: string +- Name: jvm.thread.daemon + Type: boolean +- Name: jvm.thread.state + Type: enum +- Name: system.device + Type: string +- Name: system.cpu.state + Type: enum +- Name: system.cpu.logical_number + Type: int +- Name: system.memory.state + Type: enum +- Name: system.paging.state + Type: enum +- Name: system.paging.type + Type: enum +- Name: system.paging.direction + Type: enum +- Name: system.filesystem.state + Type: enum +- Name: system.filesystem.type + Type: enum +- Name: system.filesystem.mode + Type: string +- Name: system.filesystem.mountpoint + Type: string +- Name: system.network.state + Type: enum +- Name: system.processes.status + Type: enum +- Name: client.address + Type: string +- Name: client.port + Type: int +- Name: cloud.provider + Type: enum +- Name: cloud.account.id + Type: string +- Name: cloud.region + Type: string +- Name: cloud.resource_id + Type: string +- Name: cloud.availability_zone + Type: string +- Name: cloud.platform + Type: enum +- Name: code.function + Type: string +- Name: code.namespace + Type: string +- Name: code.filepath + Type: string +- Name: code.lineno + Type: int +- Name: code.column + Type: int +- Name: code.stacktrace + Type: string +- Name: container.name + Type: string +- Name: container.id + Type: string +- Name: container.runtime + Type: string +- Name: container.image.name + Type: string +- Name: container.image.tags + Type: string[] +- Name: container.image.id + Type: string +- Name: container.image.repo_digests + Type: string[] +- Name: container.command + Type: string +- Name: container.command_line + Type: string +- Name: container.command_args + Type: string[] +- Name: container.labels + Type: template[string] +- Name: db.cassandra.coordinator.dc + Type: string +- Name: db.cassandra.coordinator.id + Type: string +- Name: db.cassandra.consistency_level + Type: enum +- Name: db.cassandra.idempotence + Type: boolean +- Name: db.cassandra.page_size + Type: int +- Name: db.cassandra.speculative_execution_count + Type: int +- Name: db.cassandra.table + Type: string +- Name: db.connection_string + Type: string +- Name: db.cosmosdb.client_id + Type: string +- Name: db.cosmosdb.connection_mode + Type: enum +- Name: db.cosmosdb.container + Type: string +- Name: db.cosmosdb.operation_type + Type: enum +- Name: db.cosmosdb.request_charge + Type: double +- Name: db.cosmosdb.request_content_length + Type: int +- Name: db.cosmosdb.status_code + Type: int +- Name: db.cosmosdb.sub_status_code + Type: int +- Name: db.elasticsearch.cluster.name + Type: string +- Name: db.elasticsearch.node.name + Type: string +- Name: db.elasticsearch.path_parts + Type: template[string] +- Name: db.jdbc.driver_classname + Type: string +- Name: db.mongodb.collection + Type: string +- Name: db.mssql.instance_name + Type: string +- Name: db.name + Type: string +- Name: db.operation + Type: string +- Name: db.redis.database_index + Type: int +- Name: db.sql.table + Type: string +- Name: db.statement + Type: string +- Name: db.system + Type: enum +- Name: db.user + Type: string +- Name: http.method + Type: string +- Name: http.status_code + Type: int +- Name: http.scheme + Type: string +- Name: http.url + Type: string +- Name: http.target + Type: string +- Name: http.request_content_length + Type: int +- Name: http.response_content_length + Type: int +- Name: http.flavor + Type: enum +- Name: http.user_agent + Type: string +- Name: net.sock.peer.name + Type: string +- Name: net.sock.peer.addr + Type: string +- Name: net.sock.peer.port + Type: int +- Name: net.peer.name + Type: string +- Name: net.peer.port + Type: int +- Name: net.host.name + Type: string +- Name: net.host.port + Type: int +- Name: net.sock.host.addr + Type: string +- Name: net.sock.host.port + Type: int +- Name: net.transport + Type: enum +- Name: net.protocol.name + Type: string +- Name: net.protocol.version + Type: string +- Name: net.sock.family + Type: enum +- Name: destination.address + Type: string +- Name: destination.port + Type: int +- Name: device.id + Type: string +- Name: device.manufacturer + Type: string +- Name: device.model.identifier + Type: string +- Name: device.model.name + Type: string +- Name: disk.io.direction + Type: enum +- Name: error.type + Type: enum +- Name: host.id + Type: string +- Name: host.name + Type: string +- Name: host.type + Type: string +- Name: host.arch + Type: enum +- Name: host.image.name + Type: string +- Name: host.image.id + Type: string +- Name: host.image.version + Type: string +- Name: host.ip + Type: string[] +- Name: host.mac + Type: string[] +- Name: host.cpu.vendor.id + Type: string +- Name: host.cpu.family + Type: string +- Name: host.cpu.model.id + Type: string +- Name: host.cpu.model.name + Type: string +- Name: host.cpu.stepping + Type: int +- Name: host.cpu.cache.l2.size + Type: int +- Name: http.request.body.size + Type: int +- Name: http.request.header + Type: template[string[]] +- Name: http.request.method + Type: enum +- Name: http.request.method_original + Type: string +- Name: http.request.resend_count + Type: int +- Name: http.response.body.size + Type: int +- Name: http.response.header + Type: template[string[]] +- Name: http.response.status_code + Type: int +- Name: http.route + Type: string +- Name: k8s.cluster.name + Type: string +- Name: k8s.cluster.uid + Type: string +- Name: k8s.node.name + Type: string +- Name: k8s.node.uid + Type: string +- Name: k8s.namespace.name + Type: string +- Name: k8s.pod.uid + Type: string +- Name: k8s.pod.name + Type: string +- Name: k8s.container.name + Type: string +- Name: k8s.container.restart_count + Type: int +- Name: k8s.replicaset.uid + Type: string +- Name: k8s.replicaset.name + Type: string +- Name: k8s.deployment.uid + Type: string +- Name: k8s.deployment.name + Type: string +- Name: k8s.statefulset.uid + Type: string +- Name: k8s.statefulset.name + Type: string +- Name: k8s.daemonset.uid + Type: string +- Name: k8s.daemonset.name + Type: string +- Name: k8s.job.uid + Type: string +- Name: k8s.job.name + Type: string +- Name: k8s.cronjob.uid + Type: string +- Name: k8s.cronjob.name + Type: string +- Name: messaging.batch.message_count + Type: int +- Name: messaging.client_id + Type: string +- Name: messaging.destination.name + Type: string +- Name: messaging.destination.template + Type: string +- Name: messaging.destination.anonymous + Type: boolean +- Name: messaging.destination.temporary + Type: boolean +- Name: messaging.destination_publish.anonymous + Type: boolean +- Name: messaging.destination_publish.name + Type: string +- Name: messaging.kafka.consumer.group + Type: string +- Name: messaging.kafka.destination.partition + Type: int +- Name: messaging.kafka.message.key + Type: string +- Name: messaging.kafka.message.offset + Type: int +- Name: messaging.kafka.message.tombstone + Type: boolean +- Name: messaging.message.conversation_id + Type: string +- Name: messaging.message.envelope.size + Type: int +- Name: messaging.message.id + Type: string +- Name: messaging.message.body.size + Type: int +- Name: messaging.operation + Type: enum +- Name: messaging.rabbitmq.destination.routing_key + Type: string +- Name: messaging.rocketmq.client_group + Type: string +- Name: messaging.rocketmq.consumption_model + Type: enum +- Name: messaging.rocketmq.message.delay_time_level + Type: int +- Name: messaging.rocketmq.message.delivery_timestamp + Type: int +- Name: messaging.rocketmq.message.group + Type: string +- Name: messaging.rocketmq.message.keys + Type: string[] +- Name: messaging.rocketmq.message.tag + Type: string +- Name: messaging.rocketmq.message.type + Type: enum +- Name: messaging.rocketmq.namespace + Type: string +- Name: messaging.gcp_pubsub.message.ordering_key + Type: string +- Name: messaging.system + Type: enum +- Name: network.carrier.icc + Type: string +- Name: network.carrier.mcc + Type: string +- Name: network.carrier.mnc + Type: string +- Name: network.carrier.name + Type: string +- Name: network.connection.subtype + Type: enum +- Name: network.connection.type + Type: enum +- Name: network.local.address + Type: string +- Name: network.local.port + Type: int +- Name: network.peer.address + Type: string +- Name: network.peer.port + Type: int +- Name: network.protocol.name + Type: string +- Name: network.protocol.version + Type: string +- Name: network.transport + Type: enum +- Name: network.type + Type: enum +- Name: network.io.direction + Type: enum +- Name: oci.manifest.digest + Type: string +- Name: os.type + Type: enum +- Name: os.description + Type: string +- Name: os.name + Type: string +- Name: os.version + Type: string +- Name: os.build_id + Type: string +- Name: process.pid + Type: int +- Name: process.parent_pid + Type: int +- Name: process.executable.name + Type: string +- Name: process.executable.path + Type: string +- Name: process.command + Type: string +- Name: process.command_line + Type: string +- Name: process.command_args + Type: string[] +- Name: process.owner + Type: string +- Name: process.runtime.name + Type: string +- Name: process.runtime.version + Type: string +- Name: process.runtime.description + Type: string +- Name: rpc.connect_rpc.error_code + Type: enum +- Name: rpc.connect_rpc.request.metadata + Type: template[string[]] +- Name: rpc.connect_rpc.response.metadata + Type: template[string[]] +- Name: rpc.grpc.status_code + Type: enum +- Name: rpc.grpc.request.metadata + Type: template[string[]] +- Name: rpc.grpc.response.metadata + Type: template[string[]] +- Name: rpc.jsonrpc.error_code + Type: int +- Name: rpc.jsonrpc.error_message + Type: string +- Name: rpc.jsonrpc.request_id + Type: string +- Name: rpc.jsonrpc.version + Type: string +- Name: rpc.method + Type: string +- Name: rpc.service + Type: string +- Name: rpc.system + Type: enum +- Name: server.address + Type: string +- Name: server.port + Type: int +- Name: source.address + Type: string +- Name: source.port + Type: int +- Name: thread.id + Type: int +- Name: thread.name + Type: string +- Name: tls.cipher + Type: string +- Name: tls.client.certificate + Type: string +- Name: tls.client.certificate_chain + Type: string[] +- Name: tls.client.hash.md5 + Type: string +- Name: tls.client.hash.sha1 + Type: string +- Name: tls.client.hash.sha256 + Type: string +- Name: tls.client.issuer + Type: string +- Name: tls.client.ja3 + Type: string +- Name: tls.client.not_after + Type: string +- Name: tls.client.not_before + Type: string +- Name: tls.client.server_name + Type: string +- Name: tls.client.subject + Type: string +- Name: tls.client.supported_ciphers + Type: string[] +- Name: tls.curve + Type: string +- Name: tls.established + Type: boolean +- Name: tls.next_protocol + Type: string +- Name: tls.protocol.name + Type: enum +- Name: tls.protocol.version + Type: string +- Name: tls.resumed + Type: boolean +- Name: tls.server.certificate + Type: string +- Name: tls.server.certificate_chain + Type: string[] +- Name: tls.server.hash.md5 + Type: string +- Name: tls.server.hash.sha1 + Type: string +- Name: tls.server.hash.sha256 + Type: string +- Name: tls.server.issuer + Type: string +- Name: tls.server.ja3s + Type: string +- Name: tls.server.not_after + Type: string +- Name: tls.server.not_before + Type: string +- Name: tls.server.subject + Type: string +- Name: url.scheme + Type: string +- Name: url.full + Type: string +- Name: url.path + Type: string +- Name: url.query + Type: string +- Name: url.fragment + Type: string +- Name: user_agent.original + Type: string +- Name: android.os.api_level + Type: string +- Name: browser.brands + Type: string[] +- Name: browser.platform + Type: string +- Name: browser.mobile + Type: boolean +- Name: browser.language + Type: string +- Name: aws.ecs.container.arn + Type: string +- Name: aws.ecs.cluster.arn + Type: string +- Name: aws.ecs.launchtype + Type: enum +- Name: aws.ecs.task.arn + Type: string +- Name: aws.ecs.task.family + Type: string +- Name: aws.ecs.task.revision + Type: string +- Name: aws.eks.cluster.arn + Type: string +- Name: aws.log.group.names + Type: string[] +- Name: aws.log.group.arns + Type: string[] +- Name: aws.log.stream.names + Type: string[] +- Name: aws.log.stream.arns + Type: string[] +- Name: gcp.cloud_run.job.execution + Type: string +- Name: gcp.cloud_run.job.task_index + Type: int +- Name: gcp.gce.instance.name + Type: string +- Name: gcp.gce.instance.hostname + Type: string +- Name: heroku.release.creation_timestamp + Type: string +- Name: heroku.release.commit + Type: string +- Name: heroku.app.id + Type: string +- Name: deployment.environment + Type: string +- Name: faas.name + Type: string +- Name: faas.version + Type: string +- Name: faas.instance + Type: string +- Name: faas.max_memory + Type: int +- Name: service.name + Type: string +- Name: service.version + Type: string +- Name: service.namespace + Type: string +- Name: service.instance.id + Type: string +- Name: telemetry.sdk.name + Type: string +- Name: telemetry.sdk.language + Type: enum +- Name: telemetry.sdk.version + Type: string +- Name: telemetry.distro.name + Type: string +- Name: telemetry.distro.version + Type: string +- Name: webengine.name + Type: string +- Name: webengine.version + Type: string +- Name: webengine.description + Type: string +- Name: otel.scope.name + Type: string +- Name: otel.scope.version + Type: string +- Name: otel.library.name + Type: string +- Name: otel.library.version + Type: string +- Name: session.id + Type: string +- Name: session.previous_id + Type: string +- Name: aws.lambda.invoked_arn + Type: string +- Name: cloudevents.event_id + Type: string +- Name: cloudevents.event_source + Type: string +- Name: cloudevents.event_spec_version + Type: string +- Name: cloudevents.event_type + Type: string +- Name: cloudevents.event_subject + Type: string +- Name: opentracing.ref_type + Type: enum +- Name: otel.status_code + Type: enum +- Name: otel.status_description + Type: string +- Name: faas.invocation_id + Type: string +- Name: faas.document.collection + Type: string +- Name: faas.document.operation + Type: enum +- Name: faas.document.time + Type: string +- Name: faas.document.name + Type: string +- Name: faas.time + Type: string +- Name: faas.cron + Type: string +- Name: faas.coldstart + Type: boolean +- Name: feature_flag.key + Type: string +- Name: feature_flag.provider_name + Type: string +- Name: feature_flag.variant + Type: string +- Name: message.type + Type: enum +- Name: message.id + Type: int +- Name: message.compressed_size + Type: int +- Name: message.uncompressed_size + Type: int +- Name: exception.escaped + Type: boolean diff --git a/internal/otelschema/gold_test.go b/internal/otelschema/gold_test.go new file mode 100644 index 00000000..3250a2e0 --- /dev/null +++ b/internal/otelschema/gold_test.go @@ -0,0 +1,15 @@ +package otelschema + +import ( + "os" + "testing" + + "github.com/go-faster/sdk/gold" +) + +func TestMain(m *testing.M) { + // Explicitly registering flags for golden files. + gold.Init() + + os.Exit(m.Run()) +} diff --git a/internal/otelschema/group.go b/internal/otelschema/group.go index d7aac38a..d00237d8 100644 --- a/internal/otelschema/group.go +++ b/internal/otelschema/group.go @@ -631,7 +631,7 @@ func (s *Type) SetGroups(val []TypeGroupsItem) { type TypeGroupsItem struct { ID string `json:"id"` Prefix OptString `json:"prefix"` - Type OptString `json:"type"` + Type TypeGroupsItemType `json:"type"` Brief OptString `json:"brief"` Note OptString `json:"note"` Attributes []TypeGroupsItemAttributesItem `json:"attributes"` @@ -648,7 +648,7 @@ func (s *TypeGroupsItem) GetPrefix() OptString { } // GetType returns the value of Type. -func (s *TypeGroupsItem) GetType() OptString { +func (s *TypeGroupsItem) GetType() TypeGroupsItemType { return s.Type } @@ -678,7 +678,7 @@ func (s *TypeGroupsItem) SetPrefix(val OptString) { } // SetType sets the value of Type. -func (s *TypeGroupsItem) SetType(val OptString) { +func (s *TypeGroupsItem) SetType(val TypeGroupsItemType) { s.Type = val } @@ -765,6 +765,61 @@ func NewAttributeTypeGroupsItemAttributesItem(v Attribute) TypeGroupsItemAttribu return s } +type TypeGroupsItemType string + +const ( + TypeGroupsItemTypeMetric TypeGroupsItemType = "metric" + TypeGroupsItemTypeSpan TypeGroupsItemType = "span" + TypeGroupsItemTypeResource TypeGroupsItemType = "resource" + TypeGroupsItemTypeAttributeGroup TypeGroupsItemType = "attribute_group" +) + +// AllValues returns all TypeGroupsItemType values. +func (TypeGroupsItemType) AllValues() []TypeGroupsItemType { + return []TypeGroupsItemType{ + TypeGroupsItemTypeMetric, + TypeGroupsItemTypeSpan, + TypeGroupsItemTypeResource, + TypeGroupsItemTypeAttributeGroup, + } +} + +// MarshalText implements encoding.TextMarshaler. +func (s TypeGroupsItemType) MarshalText() ([]byte, error) { + switch s { + case TypeGroupsItemTypeMetric: + return []byte(s), nil + case TypeGroupsItemTypeSpan: + return []byte(s), nil + case TypeGroupsItemTypeResource: + return []byte(s), nil + case TypeGroupsItemTypeAttributeGroup: + return []byte(s), nil + default: + return nil, errors.Errorf("invalid value: %q", s) + } +} + +// UnmarshalText implements encoding.TextUnmarshaler. +func (s *TypeGroupsItemType) UnmarshalText(data []byte) error { + switch TypeGroupsItemType(data) { + case TypeGroupsItemTypeMetric: + *s = TypeGroupsItemTypeMetric + return nil + case TypeGroupsItemTypeSpan: + *s = TypeGroupsItemTypeSpan + return nil + case TypeGroupsItemTypeResource: + *s = TypeGroupsItemTypeResource + return nil + case TypeGroupsItemTypeAttributeGroup: + *s = TypeGroupsItemTypeAttributeGroup + return nil + default: + return errors.Errorf("invalid value: %q", data) + } +} + // Encode implements json.Marshaler. func (s *Attribute) Encode(e *jx.Encoder) { e.ObjStart() @@ -1909,10 +1964,8 @@ func (s *TypeGroupsItem) encodeFields(e *jx.Encoder) { } } { - if s.Type.Set { - e.FieldStart("type") - s.Type.Encode(e) - } + e.FieldStart("type") + s.Type.Encode(e) } { if s.Brief.Set { @@ -1979,8 +2032,8 @@ func (s *TypeGroupsItem) Decode(d *jx.Decoder) error { return errors.Wrap(err, "decode field \"prefix\"") } case "type": + requiredBitSet[0] |= 1 << 2 if err := func() error { - s.Type.Reset() if err := s.Type.Decode(d); err != nil { return err } @@ -2035,7 +2088,7 @@ func (s *TypeGroupsItem) Decode(d *jx.Decoder) error { // Validate required fields. var failures []validate.FieldError for i, mask := range [1]uint8{ - 0b00000001, + 0b00000101, } { if result := (requiredBitSet[i] & mask) ^ mask; result != 0 { // Mask only required fields and check equality to mask using XOR. @@ -2183,6 +2236,50 @@ func (s *TypeGroupsItemAttributesItem) UnmarshalJSON(data []byte) error { return s.Decode(d) } +// Encode encodes TypeGroupsItemType as json. +func (s TypeGroupsItemType) Encode(e *jx.Encoder) { + e.Str(string(s)) +} + +// Decode decodes TypeGroupsItemType from json. +func (s *TypeGroupsItemType) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode TypeGroupsItemType to nil") + } + v, err := d.StrBytes() + if err != nil { + return err + } + // Try to use constant string. + switch TypeGroupsItemType(v) { + case TypeGroupsItemTypeMetric: + *s = TypeGroupsItemTypeMetric + case TypeGroupsItemTypeSpan: + *s = TypeGroupsItemTypeSpan + case TypeGroupsItemTypeResource: + *s = TypeGroupsItemTypeResource + case TypeGroupsItemTypeAttributeGroup: + *s = TypeGroupsItemTypeAttributeGroup + default: + *s = TypeGroupsItemType(v) + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s TypeGroupsItemType) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *TypeGroupsItemType) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + func (s *Attribute) Validate() error { if s == nil { return validate.ErrNilPointer @@ -2343,6 +2440,17 @@ func (s *TypeGroupsItem) Validate() error { } var failures []validate.FieldError + if err := func() error { + if err := s.Type.Validate(); err != nil { + return err + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: "type", + Error: err, + }) + } if err := func() error { var failures []validate.FieldError for i, elem := range s.Attributes { @@ -2387,3 +2495,18 @@ func (s TypeGroupsItemAttributesItem) Validate() error { return errors.Errorf("invalid type %q", s.Type) } } + +func (s TypeGroupsItemType) Validate() error { + switch s { + case "metric": + return nil + case "span": + return nil + case "resource": + return nil + case "attribute_group": + return nil + default: + return errors.Errorf("invalid value: %v", s) + } +} diff --git a/internal/otelschema/list_test.go b/internal/otelschema/list_test.go new file mode 100644 index 00000000..f04dad62 --- /dev/null +++ b/internal/otelschema/list_test.go @@ -0,0 +1,65 @@ +package otelschema + +import ( + "io/fs" + "os" + "path/filepath" + "testing" + + "github.com/go-faster/sdk/gold" + "github.com/stretchr/testify/require" + "sigs.k8s.io/yaml" +) + +func TestParseAllAttributes(t *testing.T) { + var parsed []TypeGroupsItem + require.NoError(t, filepath.Walk(filepath.Join("_testdata", "model"), func(path string, info fs.FileInfo, err error) error { + require.NoError(t, err) + if info.IsDir() { + return nil + } + if filepath.Ext(path) != ".yaml" { + return nil + } + data, err := os.ReadFile(path) + require.NoError(t, err) + + var schema Type + jsonData, err := yaml.YAMLToJSON(data) + require.NoError(t, err) + require.NoError(t, schema.UnmarshalJSON(jsonData)) + parsed = append(parsed, schema.Groups...) + return nil + })) + type entry struct { + Name string + Type string + } + var entries []entry + for _, group := range parsed { + for _, attr := range group.Attributes { + v, ok := attr.GetAttribute() + if !ok { + continue + } + name := v.ID + if prefix, ok := group.Prefix.Get(); ok { + name = prefix + "." + name + } + typ := "enum" + if s, ok := v.Type.GetString(); ok { + typ = s + } + t.Logf("%s (%s)", name, typ) + entries = append(entries, entry{ + Name: name, + Type: typ, + }) + } + } + t.Logf("total: %d", len(entries)) + data, err := yaml.Marshal(entries) + require.NoError(t, err) + + gold.Str(t, string(data), "all_attributes.yaml") +} diff --git a/internal/otelschema/schema.yml b/internal/otelschema/schema.yml index 342c31d7..3c5c217f 100644 --- a/internal/otelschema/schema.yml +++ b/internal/otelschema/schema.yml @@ -96,6 +96,11 @@ properties: type: string type: type: string + enum: + - metric + - span + - resource + - attribute_group brief: type: string note: @@ -108,5 +113,6 @@ properties: - $ref: "#/$defs/attribute" required: - id + - type required: - groups From 205f583b59ce41dc25517f67bd8e66221530aba6 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 12:52:33 +0300 Subject: [PATCH 050/112] fix(otelschema): make examples array --- internal/otelschema/group.go | 30 ++++++++++++++++++++++-------- internal/otelschema/schema.yml | 4 +++- 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/internal/otelschema/group.go b/internal/otelschema/group.go index d00237d8..e62df8d2 100644 --- a/internal/otelschema/group.go +++ b/internal/otelschema/group.go @@ -21,7 +21,7 @@ type Attribute struct { Brief OptString `json:"brief"` Note OptString `json:"note"` Tag OptString `json:"tag"` - Examples jx.Raw `json:"examples"` + Examples []jx.Raw `json:"examples"` RequirementLevel OptRequirementLevel `json:"requirement_level"` Stability OptString `json:"stability"` } @@ -57,7 +57,7 @@ func (s *Attribute) GetTag() OptString { } // GetExamples returns the value of Examples. -func (s *Attribute) GetExamples() jx.Raw { +func (s *Attribute) GetExamples() []jx.Raw { return s.Examples } @@ -102,7 +102,7 @@ func (s *Attribute) SetTag(val OptString) { } // SetExamples sets the value of Examples. -func (s *Attribute) SetExamples(val jx.Raw) { +func (s *Attribute) SetExamples(val []jx.Raw) { s.Examples = val } @@ -862,9 +862,15 @@ func (s *Attribute) encodeFields(e *jx.Encoder) { } } { - if len(s.Examples) != 0 { + if s.Examples != nil { e.FieldStart("examples") - e.Raw(s.Examples) + e.ArrStart() + for _, elem := range s.Examples { + if len(elem) != 0 { + e.Raw(elem) + } + } + e.ArrEnd() } } { @@ -966,9 +972,17 @@ func (s *Attribute) Decode(d *jx.Decoder) error { } case "examples": if err := func() error { - v, err := d.RawAppend(nil) - s.Examples = jx.Raw(v) - if err != nil { + s.Examples = make([]jx.Raw, 0) + if err := d.Arr(func(d *jx.Decoder) error { + var elem jx.Raw + v, err := d.RawAppend(nil) + elem = jx.Raw(v) + if err != nil { + return err + } + s.Examples = append(s.Examples, elem) + return nil + }); err != nil { return err } return nil diff --git a/internal/otelschema/schema.yml b/internal/otelschema/schema.yml index 3c5c217f..e67998ae 100644 --- a/internal/otelschema/schema.yml +++ b/internal/otelschema/schema.yml @@ -73,7 +73,9 @@ $defs: type: string tag: type: string - examples: {} + examples: + type: array + items: {} requirement_level: $ref: "#/$defs/requirementLevel" stability: From 16a430dd5b5f7d7f9894b579b27cb2022c74884f Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 12:54:16 +0300 Subject: [PATCH 051/112] fix(otelschema): make examples raw back --- internal/otelschema/group.go | 30 ++++++++---------------------- internal/otelschema/schema.yml | 4 +--- 2 files changed, 9 insertions(+), 25 deletions(-) diff --git a/internal/otelschema/group.go b/internal/otelschema/group.go index e62df8d2..d00237d8 100644 --- a/internal/otelschema/group.go +++ b/internal/otelschema/group.go @@ -21,7 +21,7 @@ type Attribute struct { Brief OptString `json:"brief"` Note OptString `json:"note"` Tag OptString `json:"tag"` - Examples []jx.Raw `json:"examples"` + Examples jx.Raw `json:"examples"` RequirementLevel OptRequirementLevel `json:"requirement_level"` Stability OptString `json:"stability"` } @@ -57,7 +57,7 @@ func (s *Attribute) GetTag() OptString { } // GetExamples returns the value of Examples. -func (s *Attribute) GetExamples() []jx.Raw { +func (s *Attribute) GetExamples() jx.Raw { return s.Examples } @@ -102,7 +102,7 @@ func (s *Attribute) SetTag(val OptString) { } // SetExamples sets the value of Examples. -func (s *Attribute) SetExamples(val []jx.Raw) { +func (s *Attribute) SetExamples(val jx.Raw) { s.Examples = val } @@ -862,15 +862,9 @@ func (s *Attribute) encodeFields(e *jx.Encoder) { } } { - if s.Examples != nil { + if len(s.Examples) != 0 { e.FieldStart("examples") - e.ArrStart() - for _, elem := range s.Examples { - if len(elem) != 0 { - e.Raw(elem) - } - } - e.ArrEnd() + e.Raw(s.Examples) } } { @@ -972,17 +966,9 @@ func (s *Attribute) Decode(d *jx.Decoder) error { } case "examples": if err := func() error { - s.Examples = make([]jx.Raw, 0) - if err := d.Arr(func(d *jx.Decoder) error { - var elem jx.Raw - v, err := d.RawAppend(nil) - elem = jx.Raw(v) - if err != nil { - return err - } - s.Examples = append(s.Examples, elem) - return nil - }); err != nil { + v, err := d.RawAppend(nil) + s.Examples = jx.Raw(v) + if err != nil { return err } return nil diff --git a/internal/otelschema/schema.yml b/internal/otelschema/schema.yml index e67998ae..3c5c217f 100644 --- a/internal/otelschema/schema.yml +++ b/internal/otelschema/schema.yml @@ -73,9 +73,7 @@ $defs: type: string tag: type: string - examples: - type: array - items: {} + examples: {} requirement_level: $ref: "#/$defs/requirementLevel" stability: From 2b12269be74ed801772fd51ac9b9ab13f1ed711c Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 13:03:42 +0300 Subject: [PATCH 052/112] feat(otelschema): improve examples schema --- internal/otelschema/group.go | 557 +++++++++++++++++++++++++----- internal/otelschema/group_test.go | 1 - internal/otelschema/schema.yml | 19 +- 3 files changed, 485 insertions(+), 92 deletions(-) diff --git a/internal/otelschema/group.go b/internal/otelschema/group.go index d00237d8..b87aa587 100644 --- a/internal/otelschema/group.go +++ b/internal/otelschema/group.go @@ -21,7 +21,7 @@ type Attribute struct { Brief OptString `json:"brief"` Note OptString `json:"note"` Tag OptString `json:"tag"` - Examples jx.Raw `json:"examples"` + Examples OptExamples `json:"examples"` RequirementLevel OptRequirementLevel `json:"requirement_level"` Stability OptString `json:"stability"` } @@ -57,7 +57,7 @@ func (s *Attribute) GetTag() OptString { } // GetExamples returns the value of Examples. -func (s *Attribute) GetExamples() jx.Raw { +func (s *Attribute) GetExamples() OptExamples { return s.Examples } @@ -102,7 +102,7 @@ func (s *Attribute) SetTag(val OptString) { } // SetExamples sets the value of Examples. -func (s *Attribute) SetExamples(val jx.Raw) { +func (s *Attribute) SetExamples(val OptExamples) { s.Examples = val } @@ -122,7 +122,7 @@ type AttributeReference struct { Brief OptString `json:"brief"` Note OptString `json:"note"` Tag OptString `json:"tag"` - Examples []jx.Raw `json:"examples"` + Examples OptExamples `json:"examples"` RequirementLevel OptRequirementLevel `json:"requirement_level"` SamplingRelevant OptBool `json:"sampling_relevant"` } @@ -148,7 +148,7 @@ func (s *AttributeReference) GetTag() OptString { } // GetExamples returns the value of Examples. -func (s *AttributeReference) GetExamples() []jx.Raw { +func (s *AttributeReference) GetExamples() OptExamples { return s.Examples } @@ -183,7 +183,7 @@ func (s *AttributeReference) SetTag(val OptString) { } // SetExamples sets the value of Examples. -func (s *AttributeReference) SetExamples(val []jx.Raw) { +func (s *AttributeReference) SetExamples(val OptExamples) { s.Examples = val } @@ -336,9 +336,9 @@ func (s *EnumMembersItem) SetBrief(val OptString) { // EnumMembersItemValue represents sum type. type EnumMembersItemValue struct { - Type EnumMembersItemValueType // switch on this field - String string - Float64 float64 + Type EnumMembersItemValueType // switch on this field + String string + Int int } // EnumMembersItemValueType is oneOf type of EnumMembersItemValue. @@ -346,15 +346,15 @@ type EnumMembersItemValueType string // Possible values for EnumMembersItemValueType. const ( - StringEnumMembersItemValue EnumMembersItemValueType = "string" - Float64EnumMembersItemValue EnumMembersItemValueType = "float64" + StringEnumMembersItemValue EnumMembersItemValueType = "string" + IntEnumMembersItemValue EnumMembersItemValueType = "int" ) // IsString reports whether EnumMembersItemValue is string. func (s EnumMembersItemValue) IsString() bool { return s.Type == StringEnumMembersItemValue } -// IsFloat64 reports whether EnumMembersItemValue is float64. -func (s EnumMembersItemValue) IsFloat64() bool { return s.Type == Float64EnumMembersItemValue } +// IsInt reports whether EnumMembersItemValue is int. +func (s EnumMembersItemValue) IsInt() bool { return s.Type == IntEnumMembersItemValue } // SetString sets EnumMembersItemValue to string. func (s *EnumMembersItemValue) SetString(v string) { @@ -377,27 +377,183 @@ func NewStringEnumMembersItemValue(v string) EnumMembersItemValue { return s } -// SetFloat64 sets EnumMembersItemValue to float64. -func (s *EnumMembersItemValue) SetFloat64(v float64) { - s.Type = Float64EnumMembersItemValue +// SetInt sets EnumMembersItemValue to int. +func (s *EnumMembersItemValue) SetInt(v int) { + s.Type = IntEnumMembersItemValue + s.Int = v +} + +// GetInt returns int and true boolean if EnumMembersItemValue is int. +func (s EnumMembersItemValue) GetInt() (v int, ok bool) { + if !s.IsInt() { + return v, false + } + return s.Int, true +} + +// NewIntEnumMembersItemValue returns new EnumMembersItemValue from int. +func NewIntEnumMembersItemValue(v int) EnumMembersItemValue { + var s EnumMembersItemValue + s.SetInt(v) + return s +} + +// Ref: #/$defs/exampleValue +// ExampleValue represents sum type. +type ExampleValue struct { + Type ExampleValueType // switch on this field + String string + Float64 float64 + Bool bool +} + +// ExampleValueType is oneOf type of ExampleValue. +type ExampleValueType string + +// Possible values for ExampleValueType. +const ( + StringExampleValue ExampleValueType = "string" + Float64ExampleValue ExampleValueType = "float64" + BoolExampleValue ExampleValueType = "bool" +) + +// IsString reports whether ExampleValue is string. +func (s ExampleValue) IsString() bool { return s.Type == StringExampleValue } + +// IsFloat64 reports whether ExampleValue is float64. +func (s ExampleValue) IsFloat64() bool { return s.Type == Float64ExampleValue } + +// IsBool reports whether ExampleValue is bool. +func (s ExampleValue) IsBool() bool { return s.Type == BoolExampleValue } + +// SetString sets ExampleValue to string. +func (s *ExampleValue) SetString(v string) { + s.Type = StringExampleValue + s.String = v +} + +// GetString returns string and true boolean if ExampleValue is string. +func (s ExampleValue) GetString() (v string, ok bool) { + if !s.IsString() { + return v, false + } + return s.String, true +} + +// NewStringExampleValue returns new ExampleValue from string. +func NewStringExampleValue(v string) ExampleValue { + var s ExampleValue + s.SetString(v) + return s +} + +// SetFloat64 sets ExampleValue to float64. +func (s *ExampleValue) SetFloat64(v float64) { + s.Type = Float64ExampleValue s.Float64 = v } -// GetFloat64 returns float64 and true boolean if EnumMembersItemValue is float64. -func (s EnumMembersItemValue) GetFloat64() (v float64, ok bool) { +// GetFloat64 returns float64 and true boolean if ExampleValue is float64. +func (s ExampleValue) GetFloat64() (v float64, ok bool) { if !s.IsFloat64() { return v, false } return s.Float64, true } -// NewFloat64EnumMembersItemValue returns new EnumMembersItemValue from float64. -func NewFloat64EnumMembersItemValue(v float64) EnumMembersItemValue { - var s EnumMembersItemValue +// NewFloat64ExampleValue returns new ExampleValue from float64. +func NewFloat64ExampleValue(v float64) ExampleValue { + var s ExampleValue s.SetFloat64(v) return s } +// SetBool sets ExampleValue to bool. +func (s *ExampleValue) SetBool(v bool) { + s.Type = BoolExampleValue + s.Bool = v +} + +// GetBool returns bool and true boolean if ExampleValue is bool. +func (s ExampleValue) GetBool() (v bool, ok bool) { + if !s.IsBool() { + return v, false + } + return s.Bool, true +} + +// NewBoolExampleValue returns new ExampleValue from bool. +func NewBoolExampleValue(v bool) ExampleValue { + var s ExampleValue + s.SetBool(v) + return s +} + +// Ref: #/$defs/examples +// Examples represents sum type. +type Examples struct { + Type ExamplesType // switch on this field + ExampleValueArray []ExampleValue + ExampleValue ExampleValue +} + +// ExamplesType is oneOf type of Examples. +type ExamplesType string + +// Possible values for ExamplesType. +const ( + ExampleValueArrayExamples ExamplesType = "[]ExampleValue" + ExampleValueExamples ExamplesType = "ExampleValue" +) + +// IsExampleValueArray reports whether Examples is []ExampleValue. +func (s Examples) IsExampleValueArray() bool { return s.Type == ExampleValueArrayExamples } + +// IsExampleValue reports whether Examples is ExampleValue. +func (s Examples) IsExampleValue() bool { return s.Type == ExampleValueExamples } + +// SetExampleValueArray sets Examples to []ExampleValue. +func (s *Examples) SetExampleValueArray(v []ExampleValue) { + s.Type = ExampleValueArrayExamples + s.ExampleValueArray = v +} + +// GetExampleValueArray returns []ExampleValue and true boolean if Examples is []ExampleValue. +func (s Examples) GetExampleValueArray() (v []ExampleValue, ok bool) { + if !s.IsExampleValueArray() { + return v, false + } + return s.ExampleValueArray, true +} + +// NewExampleValueArrayExamples returns new Examples from []ExampleValue. +func NewExampleValueArrayExamples(v []ExampleValue) Examples { + var s Examples + s.SetExampleValueArray(v) + return s +} + +// SetExampleValue sets Examples to ExampleValue. +func (s *Examples) SetExampleValue(v ExampleValue) { + s.Type = ExampleValueExamples + s.ExampleValue = v +} + +// GetExampleValue returns ExampleValue and true boolean if Examples is ExampleValue. +func (s Examples) GetExampleValue() (v ExampleValue, ok bool) { + if !s.IsExampleValue() { + return v, false + } + return s.ExampleValue, true +} + +// NewExampleValueExamples returns new Examples from ExampleValue. +func NewExampleValueExamples(v ExampleValue) Examples { + var s Examples + s.SetExampleValue(v) + return s +} + // NewOptBool returns new OptBool with value set to v. func NewOptBool(v bool) OptBool { return OptBool{ @@ -444,6 +600,52 @@ func (o OptBool) Or(d bool) bool { return d } +// NewOptExamples returns new OptExamples with value set to v. +func NewOptExamples(v Examples) OptExamples { + return OptExamples{ + Value: v, + Set: true, + } +} + +// OptExamples is optional Examples. +type OptExamples struct { + Value Examples + Set bool +} + +// IsSet returns true if OptExamples was set. +func (o OptExamples) IsSet() bool { return o.Set } + +// Reset unsets value. +func (o *OptExamples) Reset() { + var v Examples + o.Value = v + o.Set = false +} + +// SetTo sets value to v. +func (o *OptExamples) SetTo(v Examples) { + o.Set = true + o.Value = v +} + +// Get returns value and boolean that denotes whether value was set. +func (o OptExamples) Get() (v Examples, ok bool) { + if !o.Set { + return v, false + } + return o.Value, true +} + +// Or returns value if set, or given parameter if does not. +func (o OptExamples) Or(d Examples) Examples { + if v, ok := o.Get(); ok { + return v + } + return d +} + // NewOptRequirementLevel returns new OptRequirementLevel with value set to v. func NewOptRequirementLevel(v RequirementLevel) OptRequirementLevel { return OptRequirementLevel{ @@ -862,9 +1064,9 @@ func (s *Attribute) encodeFields(e *jx.Encoder) { } } { - if len(s.Examples) != 0 { + if s.Examples.Set { e.FieldStart("examples") - e.Raw(s.Examples) + s.Examples.Encode(e) } } { @@ -966,9 +1168,8 @@ func (s *Attribute) Decode(d *jx.Decoder) error { } case "examples": if err := func() error { - v, err := d.RawAppend(nil) - s.Examples = jx.Raw(v) - if err != nil { + s.Examples.Reset() + if err := s.Examples.Decode(d); err != nil { return err } return nil @@ -1084,15 +1285,9 @@ func (s *AttributeReference) encodeFields(e *jx.Encoder) { } } { - if s.Examples != nil { + if s.Examples.Set { e.FieldStart("examples") - e.ArrStart() - for _, elem := range s.Examples { - if len(elem) != 0 { - e.Raw(elem) - } - } - e.ArrEnd() + s.Examples.Encode(e) } } { @@ -1172,17 +1367,8 @@ func (s *AttributeReference) Decode(d *jx.Decoder) error { } case "examples": if err := func() error { - s.Examples = make([]jx.Raw, 0) - if err := d.Arr(func(d *jx.Decoder) error { - var elem jx.Raw - v, err := d.RawAppend(nil) - elem = jx.Raw(v) - if err != nil { - return err - } - s.Examples = append(s.Examples, elem) - return nil - }); err != nil { + s.Examples.Reset() + if err := s.Examples.Decode(d); err != nil { return err } return nil @@ -1586,8 +1772,8 @@ func (s EnumMembersItemValue) Encode(e *jx.Encoder) { switch s.Type { case StringEnumMembersItemValue: e.Str(s.String) - case Float64EnumMembersItemValue: - e.Float64(s.Float64) + case IntEnumMembersItemValue: + e.Int(s.Int) } } @@ -1599,12 +1785,12 @@ func (s *EnumMembersItemValue) Decode(d *jx.Decoder) error { // Sum type type_discriminator. switch t := d.Next(); t { case jx.Number: - v, err := d.Float64() - s.Float64 = float64(v) + v, err := d.Int() + s.Int = int(v) if err != nil { return err } - s.Type = Float64EnumMembersItemValue + s.Type = IntEnumMembersItemValue case jx.String: v, err := d.Str() s.String = string(v) @@ -1631,6 +1817,123 @@ func (s *EnumMembersItemValue) UnmarshalJSON(data []byte) error { return s.Decode(d) } +// Encode encodes ExampleValue as json. +func (s ExampleValue) Encode(e *jx.Encoder) { + switch s.Type { + case StringExampleValue: + e.Str(s.String) + case Float64ExampleValue: + e.Float64(s.Float64) + case BoolExampleValue: + e.Bool(s.Bool) + } +} + +// Decode decodes ExampleValue from json. +func (s *ExampleValue) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode ExampleValue to nil") + } + // Sum type type_discriminator. + switch t := d.Next(); t { + case jx.Bool: + v, err := d.Bool() + s.Bool = bool(v) + if err != nil { + return err + } + s.Type = BoolExampleValue + case jx.Number: + v, err := d.Float64() + s.Float64 = float64(v) + if err != nil { + return err + } + s.Type = Float64ExampleValue + case jx.String: + v, err := d.Str() + s.String = string(v) + if err != nil { + return err + } + s.Type = StringExampleValue + default: + return errors.Errorf("unexpected json type %q", t) + } + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s ExampleValue) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *ExampleValue) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + +// Encode encodes Examples as json. +func (s Examples) Encode(e *jx.Encoder) { + switch s.Type { + case ExampleValueArrayExamples: + e.ArrStart() + for _, elem := range s.ExampleValueArray { + elem.Encode(e) + } + e.ArrEnd() + case ExampleValueExamples: + s.ExampleValue.Encode(e) + } +} + +// Decode decodes Examples from json. +func (s *Examples) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode Examples to nil") + } + // Sum type type_discriminator. + switch t := d.Next(); t { + case jx.Array: + s.ExampleValueArray = make([]ExampleValue, 0) + if err := d.Arr(func(d *jx.Decoder) error { + var elem ExampleValue + if err := elem.Decode(d); err != nil { + return err + } + s.ExampleValueArray = append(s.ExampleValueArray, elem) + return nil + }); err != nil { + return err + } + s.Type = ExampleValueArrayExamples + case jx.Bool, jx.Number, jx.String: + if err := s.ExampleValue.Decode(d); err != nil { + return err + } + s.Type = ExampleValueExamples + default: + return errors.Errorf("unexpected json type %q", t) + } + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s Examples) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *Examples) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + // Encode encodes bool as json. func (o OptBool) Encode(e *jx.Encoder) { if !o.Set { @@ -1666,6 +1969,39 @@ func (s *OptBool) UnmarshalJSON(data []byte) error { return s.Decode(d) } +// Encode encodes Examples as json. +func (o OptExamples) Encode(e *jx.Encoder) { + if !o.Set { + return + } + o.Value.Encode(e) +} + +// Decode decodes Examples from json. +func (o *OptExamples) Decode(d *jx.Decoder) error { + if o == nil { + return errors.New("invalid: unable to decode OptExamples to nil") + } + o.Set = true + if err := o.Value.Decode(d); err != nil { + return err + } + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s OptExamples) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *OptExamples) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + // Encode encodes RequirementLevel as json. func (o OptRequirementLevel) Encode(e *jx.Encoder) { if !o.Set { @@ -2297,57 +2633,51 @@ func (s *Attribute) Validate() error { Error: err, }) } + if err := func() error { + if value, ok := s.Examples.Get(); ok { + if err := func() error { + if err := value.Validate(); err != nil { + return err + } + return nil + }(); err != nil { + return err + } + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: "examples", + Error: err, + }) + } if len(failures) > 0 { return &validate.Error{Fields: failures} } return nil } -func (s AttributeType) Validate() error { - switch s.Type { - case EnumAttributeType: - if err := s.Enum.Validate(); err != nil { - return err - } - return nil - case StringAttributeType: - return nil // no validation needed - default: - return errors.Errorf("invalid type %q", s.Type) - } -} - -func (s *Enum) Validate() error { +func (s *AttributeReference) Validate() error { if s == nil { return validate.ErrNilPointer } var failures []validate.FieldError if err := func() error { - if s.Members == nil { - return errors.New("nil is invalid value") - } - var failures []validate.FieldError - for i, elem := range s.Members { + if value, ok := s.Examples.Get(); ok { if err := func() error { - if err := elem.Validate(); err != nil { + if err := value.Validate(); err != nil { return err } return nil }(); err != nil { - failures = append(failures, validate.FieldError{ - Name: fmt.Sprintf("[%d]", i), - Error: err, - }) + return err } } - if len(failures) > 0 { - return &validate.Error{Fields: failures} - } return nil }(); err != nil { failures = append(failures, validate.FieldError{ - Name: "members", + Name: "examples", Error: err, }) } @@ -2357,20 +2687,34 @@ func (s *Enum) Validate() error { return nil } -func (s *EnumMembersItem) Validate() error { +func (s AttributeType) Validate() error { + switch s.Type { + case EnumAttributeType: + if err := s.Enum.Validate(); err != nil { + return err + } + return nil + case StringAttributeType: + return nil // no validation needed + default: + return errors.Errorf("invalid type %q", s.Type) + } +} + +func (s *Enum) Validate() error { if s == nil { return validate.ErrNilPointer } var failures []validate.FieldError if err := func() error { - if err := s.Value.Validate(); err != nil { - return err + if s.Members == nil { + return errors.New("nil is invalid value") } return nil }(); err != nil { failures = append(failures, validate.FieldError{ - Name: "value", + Name: "members", Error: err, }) } @@ -2380,15 +2724,51 @@ func (s *EnumMembersItem) Validate() error { return nil } -func (s EnumMembersItemValue) Validate() error { +func (s ExampleValue) Validate() error { switch s.Type { - case StringEnumMembersItemValue: + case StringExampleValue: return nil // no validation needed - case Float64EnumMembersItemValue: + case Float64ExampleValue: if err := (validate.Float{}).Validate(float64(s.Float64)); err != nil { return errors.Wrap(err, "float") } return nil + case BoolExampleValue: + return nil // no validation needed + default: + return errors.Errorf("invalid type %q", s.Type) + } +} + +func (s Examples) Validate() error { + switch s.Type { + case ExampleValueArrayExamples: + if s.ExampleValueArray == nil { + return errors.New("nil is invalid value") + } + var failures []validate.FieldError + for i, elem := range s.ExampleValueArray { + if err := func() error { + if err := elem.Validate(); err != nil { + return err + } + return nil + }(); err != nil { + failures = append(failures, validate.FieldError{ + Name: fmt.Sprintf("[%d]", i), + Error: err, + }) + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + return nil + case ExampleValueExamples: + if err := s.ExampleValue.Validate(); err != nil { + return err + } + return nil default: return errors.Errorf("invalid type %q", s.Type) } @@ -2485,7 +2865,10 @@ func (s *TypeGroupsItem) Validate() error { func (s TypeGroupsItemAttributesItem) Validate() error { switch s.Type { case AttributeReferenceTypeGroupsItemAttributesItem: - return nil // no validation needed + if err := s.AttributeReference.Validate(); err != nil { + return err + } + return nil case AttributeTypeGroupsItemAttributesItem: if err := s.Attribute.Validate(); err != nil { return err diff --git a/internal/otelschema/group_test.go b/internal/otelschema/group_test.go index 98573e17..c52e5764 100644 --- a/internal/otelschema/group_test.go +++ b/internal/otelschema/group_test.go @@ -26,7 +26,6 @@ func TestParse(t *testing.T) { var schema Type jsonData, err := yaml.YAMLToJSON(data) require.NoError(t, err) - t.Logf("json: %s", jsonData) require.NoError(t, schema.UnmarshalJSON(jsonData)) }) return nil diff --git a/internal/otelschema/schema.yml b/internal/otelschema/schema.yml index 3c5c217f..d1cff300 100644 --- a/internal/otelschema/schema.yml +++ b/internal/otelschema/schema.yml @@ -1,6 +1,17 @@ $schema: "http://json-schema.org/draft-07/schema#" $defs: + exampleValue: + oneOf: + - type: string + - type: number + - type: boolean + examples: + oneOf: + - type: array + items: + $ref: "#/$defs/exampleValue" + - $ref: "#/$defs/exampleValue" enum: additionalProperties: false type: object @@ -22,7 +33,7 @@ $defs: value: oneOf: - type: string - - type: number + - type: integer brief: type: string required: @@ -49,8 +60,7 @@ $defs: tag: type: string examples: - type: array - items: {} + $ref: "#/$defs/examples" requirement_level: $ref: "#/$defs/requirementLevel" sampling_relevant: @@ -73,7 +83,8 @@ $defs: type: string tag: type: string - examples: {} + examples: + $ref: "#/$defs/examples" requirement_level: $ref: "#/$defs/requirementLevel" stability: From d22ddd7920f64278eb9d1c48ba6d98651d2405cd Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 13:13:34 +0300 Subject: [PATCH 053/112] test(otelschema): improve all attributes golden file --- .../otelschema/_golden/all_attributes.yaml | 282 +++++++++++++----- internal/otelschema/list_test.go | 22 +- 2 files changed, 229 insertions(+), 75 deletions(-) diff --git a/internal/otelschema/_golden/all_attributes.yaml b/internal/otelschema/_golden/all_attributes.yaml index 82b9f002..5a077dc9 100644 --- a/internal/otelschema/_golden/all_attributes.yaml +++ b/internal/otelschema/_golden/all_attributes.yaml @@ -4,12 +4,18 @@ Type: string - Name: exception.stacktrace Type: string -- Name: faas.trigger - Type: enum +- Enum: + - datasource + - http + - pubsub + - timer + - other + Name: faas.trigger + Type: string - Name: faas.invoked_name Type: string - Name: faas.invoked_provider - Type: enum + Type: string - Name: faas.invoked_region Type: string - Name: peer.service @@ -24,8 +30,11 @@ Type: string - Name: log.record.uid Type: string -- Name: log.iostream - Type: enum +- Enum: + - stdout + - stderr + Name: log.iostream + Type: string - Name: log.file.name Type: string - Name: log.file.path @@ -34,18 +43,34 @@ Type: string - Name: log.file.path_resolved Type: string -- Name: ios.state - Type: enum -- Name: android.state - Type: enum -- Name: state - Type: enum +- Enum: + - active + - inactive + - background + - foreground + - terminate + Name: ios.state + Type: string +- Enum: + - created + - background + - foreground + Name: android.state + Type: string +- Enum: + - idle + - used + Name: state + Type: string - Name: pool.name Type: string - Name: jvm.buffer.pool.name Type: string -- Name: jvm.memory.type - Type: enum +- Enum: + - heap + - non_heap + Name: jvm.memory.type + Type: string - Name: jvm.memory.pool.name Type: string - Name: jvm.gc.name @@ -54,40 +79,73 @@ Type: string - Name: jvm.thread.daemon Type: boolean -- Name: jvm.thread.state - Type: enum +- Enum: + - new + - runnable + - blocked + - waiting + - timed_waiting + - terminated + Name: jvm.thread.state + Type: string - Name: system.device Type: string - Name: system.cpu.state - Type: enum + Type: string - Name: system.cpu.logical_number Type: int - Name: system.memory.state - Type: enum -- Name: system.paging.state - Type: enum -- Name: system.paging.type - Type: enum -- Name: system.paging.direction - Type: enum -- Name: system.filesystem.state - Type: enum + Type: string +- Enum: + - used + - free + Name: system.paging.state + Type: string +- Enum: + - major + - minor + Name: system.paging.type + Type: string +- Enum: + - in + - out + Name: system.paging.direction + Type: string +- Enum: + - used + - free + - reserved + Name: system.filesystem.state + Type: string - Name: system.filesystem.type - Type: enum + Type: string - Name: system.filesystem.mode Type: string - Name: system.filesystem.mountpoint Type: string -- Name: system.network.state - Type: enum +- Enum: + - close + - close_wait + - closing + - delete + - established + - fin_wait_1 + - fin_wait_2 + - last_ack + - listen + - syn_recv + - syn_sent + - time_wait + Name: system.network.state + Type: string - Name: system.processes.status - Type: enum + Type: string - Name: client.address Type: string - Name: client.port Type: int - Name: cloud.provider - Type: enum + Type: string - Name: cloud.account.id Type: string - Name: cloud.region @@ -97,7 +155,7 @@ - Name: cloud.availability_zone Type: string - Name: cloud.platform - Type: enum + Type: string - Name: code.function Type: string - Name: code.namespace @@ -136,8 +194,20 @@ Type: string - Name: db.cassandra.coordinator.id Type: string -- Name: db.cassandra.consistency_level - Type: enum +- Enum: + - all + - each_quorum + - quorum + - local_quorum + - one + - two + - three + - local_one + - any + - serial + - local_serial + Name: db.cassandra.consistency_level + Type: string - Name: db.cassandra.idempotence Type: boolean - Name: db.cassandra.page_size @@ -150,12 +220,15 @@ Type: string - Name: db.cosmosdb.client_id Type: string -- Name: db.cosmosdb.connection_mode - Type: enum +- Enum: + - gateway + - direct + Name: db.cosmosdb.connection_mode + Type: string - Name: db.cosmosdb.container Type: string - Name: db.cosmosdb.operation_type - Type: enum + Type: string - Name: db.cosmosdb.request_charge Type: double - Name: db.cosmosdb.request_content_length @@ -187,7 +260,7 @@ - Name: db.statement Type: string - Name: db.system - Type: enum + Type: string - Name: db.user Type: string - Name: http.method @@ -205,7 +278,7 @@ - Name: http.response_content_length Type: int - Name: http.flavor - Type: enum + Type: string - Name: http.user_agent Type: string - Name: net.sock.peer.name @@ -227,13 +300,13 @@ - Name: net.sock.host.port Type: int - Name: net.transport - Type: enum + Type: string - Name: net.protocol.name Type: string - Name: net.protocol.version Type: string - Name: net.sock.family - Type: enum + Type: string - Name: destination.address Type: string - Name: destination.port @@ -246,10 +319,13 @@ Type: string - Name: device.model.name Type: string -- Name: disk.io.direction - Type: enum +- Enum: + - read + - write + Name: disk.io.direction + Type: string - Name: error.type - Type: enum + Type: string - Name: host.id Type: string - Name: host.name @@ -257,7 +333,7 @@ - Name: host.type Type: string - Name: host.arch - Type: enum + Type: string - Name: host.image.name Type: string - Name: host.image.id @@ -285,7 +361,7 @@ - Name: http.request.header Type: template[string[]] - Name: http.request.method - Type: enum + Type: string - Name: http.request.method_original Type: string - Name: http.request.resend_count @@ -375,13 +451,16 @@ - Name: messaging.message.body.size Type: int - Name: messaging.operation - Type: enum + Type: string - Name: messaging.rabbitmq.destination.routing_key Type: string - Name: messaging.rocketmq.client_group Type: string -- Name: messaging.rocketmq.consumption_model - Type: enum +- Enum: + - clustering + - broadcasting + Name: messaging.rocketmq.consumption_model + Type: string - Name: messaging.rocketmq.message.delay_time_level Type: int - Name: messaging.rocketmq.message.delivery_timestamp @@ -392,14 +471,19 @@ Type: string[] - Name: messaging.rocketmq.message.tag Type: string -- Name: messaging.rocketmq.message.type - Type: enum +- Enum: + - normal + - fifo + - delay + - transaction + Name: messaging.rocketmq.message.type + Type: string - Name: messaging.rocketmq.namespace Type: string - Name: messaging.gcp_pubsub.message.ordering_key Type: string - Name: messaging.system - Type: enum + Type: string - Name: network.carrier.icc Type: string - Name: network.carrier.mcc @@ -409,9 +493,9 @@ - Name: network.carrier.name Type: string - Name: network.connection.subtype - Type: enum + Type: string - Name: network.connection.type - Type: enum + Type: string - Name: network.local.address Type: string - Name: network.local.port @@ -425,15 +509,18 @@ - Name: network.protocol.version Type: string - Name: network.transport - Type: enum + Type: string - Name: network.type - Type: enum -- Name: network.io.direction - Type: enum + Type: string +- Enum: + - transmit + - receive + Name: network.io.direction + Type: string - Name: oci.manifest.digest Type: string - Name: os.type - Type: enum + Type: string - Name: os.description Type: string - Name: os.name @@ -464,14 +551,49 @@ Type: string - Name: process.runtime.description Type: string -- Name: rpc.connect_rpc.error_code - Type: enum +- Enum: + - cancelled + - unknown + - invalid_argument + - deadline_exceeded + - not_found + - already_exists + - permission_denied + - resource_exhausted + - failed_precondition + - aborted + - out_of_range + - unimplemented + - internal + - unavailable + - data_loss + - unauthenticated + Name: rpc.connect_rpc.error_code + Type: string - Name: rpc.connect_rpc.request.metadata Type: template[string[]] - Name: rpc.connect_rpc.response.metadata Type: template[string[]] -- Name: rpc.grpc.status_code - Type: enum +- Enum: + - 0 + - 1 + - 2 + - 3 + - 4 + - 5 + - 6 + - 7 + - 8 + - 9 + - 10 + - 11 + - 12 + - 13 + - 14 + - 15 + - 16 + Name: rpc.grpc.status_code + Type: int - Name: rpc.grpc.request.metadata Type: template[string[]] - Name: rpc.grpc.response.metadata @@ -489,7 +611,7 @@ - Name: rpc.service Type: string - Name: rpc.system - Type: enum + Type: string - Name: server.address Type: string - Name: server.port @@ -535,7 +657,7 @@ - Name: tls.next_protocol Type: string - Name: tls.protocol.name - Type: enum + Type: string - Name: tls.protocol.version Type: string - Name: tls.resumed @@ -586,8 +708,11 @@ Type: string - Name: aws.ecs.cluster.arn Type: string -- Name: aws.ecs.launchtype - Type: enum +- Enum: + - ec2 + - fargate + Name: aws.ecs.launchtype + Type: string - Name: aws.ecs.task.arn Type: string - Name: aws.ecs.task.family @@ -639,7 +764,7 @@ - Name: telemetry.sdk.name Type: string - Name: telemetry.sdk.language - Type: enum + Type: string - Name: telemetry.sdk.version Type: string - Name: telemetry.distro.name @@ -676,10 +801,16 @@ Type: string - Name: cloudevents.event_subject Type: string -- Name: opentracing.ref_type - Type: enum -- Name: otel.status_code - Type: enum +- Enum: + - child_of + - follows_from + Name: opentracing.ref_type + Type: string +- Enum: + - OK + - ERROR + Name: otel.status_code + Type: string - Name: otel.status_description Type: string - Name: faas.invocation_id @@ -687,7 +818,7 @@ - Name: faas.document.collection Type: string - Name: faas.document.operation - Type: enum + Type: string - Name: faas.document.time Type: string - Name: faas.document.name @@ -704,8 +835,11 @@ Type: string - Name: feature_flag.variant Type: string -- Name: message.type - Type: enum +- Enum: + - SENT + - RECEIVED + Name: message.type + Type: string - Name: message.id Type: int - Name: message.compressed_size diff --git a/internal/otelschema/list_test.go b/internal/otelschema/list_test.go index f04dad62..3021b395 100644 --- a/internal/otelschema/list_test.go +++ b/internal/otelschema/list_test.go @@ -34,6 +34,7 @@ func TestParseAllAttributes(t *testing.T) { type entry struct { Name string Type string + Enum []interface{} `json:"Enum,omitempty"` } var entries []entry for _, group := range parsed { @@ -46,14 +47,33 @@ func TestParseAllAttributes(t *testing.T) { if prefix, ok := group.Prefix.Get(); ok { name = prefix + "." + name } - typ := "enum" + var typ string if s, ok := v.Type.GetString(); ok { typ = s } + var enum []interface{} + if e, ok := v.Type.GetEnum(); ok { + typ = "enum" + for _, m := range e.Members { + switch m.Value.Type { + case StringEnumMembersItemValue: + enum = append(enum, m.Value.String) + typ = "string" + case IntEnumMembersItemValue: + enum = append(enum, m.Value.Int) + typ = "int" + } + } + if e.AllowCustomValues.Value { + // Not actually an enum? + enum = nil + } + } t.Logf("%s (%s)", name, typ) entries = append(entries, entry{ Name: name, Type: typ, + Enum: enum, }) } } From f31a1a45cfb9127c9dff23a7ca421c8f4bdb33cd Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 14:47:40 +0300 Subject: [PATCH 054/112] feat(otelschema): add clickhouse column types --- .gitmodules | 3 + .../otelschema/_golden/all_attributes.yaml | 850 ------ internal/otelschema/_golden/registry.yaml | 2581 +++++++++++++++++ internal/otelschema/_testdata/model/README.md | 41 - .../otelschema/_testdata/model/exception.yaml | 33 - .../_testdata/model/faas-common.yaml | 77 - .../otelschema/_testdata/model/general.yaml | 85 - .../_testdata/model/http-common.yaml | 87 - .../_testdata/model/logs/events.yaml | 18 - .../_testdata/model/logs/general.yaml | 19 - .../_testdata/model/logs/log-exception.yaml | 16 - .../model/logs/log-feature_flag.yaml | 11 - .../_testdata/model/logs/media.yaml | 49 - .../_testdata/model/logs/mobile-events.yaml | 72 - .../model/metrics/database-metrics.yaml | 107 - .../_testdata/model/metrics/faas-metrics.yaml | 81 - .../_testdata/model/metrics/http.yaml | 119 - .../metrics/jvm-metrics-experimental.yaml | 70 - .../_testdata/model/metrics/jvm-metrics.yaml | 168 -- .../_testdata/model/metrics/rpc-metrics.yaml | 122 - .../model/metrics/system-metrics.yaml | 501 ---- .../otelschema/_testdata/model/network.yaml | 28 - .../_testdata/model/registry/client.yaml | 28 - .../_testdata/model/registry/cloud.yaml | 179 -- .../_testdata/model/registry/code.yaml | 41 - .../_testdata/model/registry/container.yaml | 86 - .../_testdata/model/registry/db.yaml | 424 --- .../model/registry/deprecated/http.yaml | 71 - .../model/registry/deprecated/network.yaml | 100 - .../_testdata/model/registry/destination.yaml | 23 - .../_testdata/model/registry/device.yaml | 46 - .../_testdata/model/registry/disk.yaml | 17 - .../_testdata/model/registry/error.yaml | 35 - .../_testdata/model/registry/host.yaml | 122 - .../_testdata/model/registry/http.yaml | 135 - .../_testdata/model/registry/k8s.yaml | 140 - .../_testdata/model/registry/messaging.yaml | 245 -- .../_testdata/model/registry/network.yaml | 194 -- .../_testdata/model/registry/oci.yaml | 21 - .../_testdata/model/registry/os.yaml | 69 - .../_testdata/model/registry/process.yaml | 78 - .../_testdata/model/registry/rpc.yaml | 190 -- .../_testdata/model/registry/server.yaml | 28 - .../_testdata/model/registry/source.yaml | 23 - .../_testdata/model/registry/thread.yaml | 17 - .../_testdata/model/registry/tls.yaml | 165 -- .../_testdata/model/registry/url.yaml | 41 - .../_testdata/model/registry/user-agent.yaml | 13 - .../_testdata/model/resource/android.yaml | 14 - .../_testdata/model/resource/browser.yaml | 56 - .../_testdata/model/resource/cloud.yaml | 13 - .../resource/cloud_provider/aws/ecs.yaml | 42 - .../resource/cloud_provider/aws/eks.yaml | 12 - .../resource/cloud_provider/aws/logs.yaml | 39 - .../cloud_provider/gcp/cloud_run.yaml | 23 - .../resource/cloud_provider/gcp/gce.yaml | 22 - .../model/resource/cloud_provider/heroku.yaml | 25 - .../_testdata/model/resource/container.yaml | 22 - .../resource/deployment_environment.yaml | 21 - .../_testdata/model/resource/device.yaml | 11 - .../_testdata/model/resource/faas.yaml | 63 - .../_testdata/model/resource/host.yaml | 37 - .../_testdata/model/resource/k8s.yaml | 98 - .../_testdata/model/resource/os.yaml | 16 - .../_testdata/model/resource/process.yaml | 42 - .../_testdata/model/resource/service.yaml | 23 - .../model/resource/service_experimental.yaml | 37 - .../_testdata/model/resource/telemetry.yaml | 57 - .../resource/telemetry_experimental.yaml | 20 - .../_testdata/model/resource/webengine.yaml | 23 - .../model/scope/exporter/exporter.yaml | 30 - .../otelschema/_testdata/model/session.yaml | 26 - .../_testdata/model/trace/aws/lambda.yaml | 14 - .../_testdata/model/trace/cloudevents.yaml | 36 - .../_testdata/model/trace/compatibility.yaml | 21 - .../_testdata/model/trace/database.yaml | 259 -- .../model/trace/exporter/exporter.yaml | 21 - .../_testdata/model/trace/faas.yaml | 144 - .../_testdata/model/trace/feature-flag.yaml | 34 - .../_testdata/model/trace/http.yaml | 91 - .../model/trace/instrumentation/aws-sdk.yml | 516 ---- .../model/trace/instrumentation/graphql.yml | 32 - .../_testdata/model/trace/messaging.yaml | 182 -- .../otelschema/_testdata/model/trace/rpc.yaml | 129 - .../model/trace/trace-exception.yaml | 38 - internal/otelschema/_testdata/model/url.yaml | 13 - .../_testdata/model/version.properties | 1 - .../otelschema/_testdata/semantic-conventions | 1 + internal/otelschema/group_test.go | 6 +- internal/otelschema/list_test.go | 85 - internal/otelschema/registry_test.go | 208 ++ 91 files changed, 2798 insertions(+), 7474 deletions(-) create mode 100644 .gitmodules delete mode 100644 internal/otelschema/_golden/all_attributes.yaml create mode 100644 internal/otelschema/_golden/registry.yaml delete mode 100644 internal/otelschema/_testdata/model/README.md delete mode 100644 internal/otelschema/_testdata/model/exception.yaml delete mode 100644 internal/otelschema/_testdata/model/faas-common.yaml delete mode 100644 internal/otelschema/_testdata/model/general.yaml delete mode 100644 internal/otelschema/_testdata/model/http-common.yaml delete mode 100644 internal/otelschema/_testdata/model/logs/events.yaml delete mode 100644 internal/otelschema/_testdata/model/logs/general.yaml delete mode 100644 internal/otelschema/_testdata/model/logs/log-exception.yaml delete mode 100644 internal/otelschema/_testdata/model/logs/log-feature_flag.yaml delete mode 100644 internal/otelschema/_testdata/model/logs/media.yaml delete mode 100644 internal/otelschema/_testdata/model/logs/mobile-events.yaml delete mode 100644 internal/otelschema/_testdata/model/metrics/database-metrics.yaml delete mode 100644 internal/otelschema/_testdata/model/metrics/faas-metrics.yaml delete mode 100644 internal/otelschema/_testdata/model/metrics/http.yaml delete mode 100644 internal/otelschema/_testdata/model/metrics/jvm-metrics-experimental.yaml delete mode 100644 internal/otelschema/_testdata/model/metrics/jvm-metrics.yaml delete mode 100644 internal/otelschema/_testdata/model/metrics/rpc-metrics.yaml delete mode 100644 internal/otelschema/_testdata/model/metrics/system-metrics.yaml delete mode 100644 internal/otelschema/_testdata/model/network.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/client.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/cloud.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/code.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/container.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/db.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/deprecated/http.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/deprecated/network.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/destination.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/device.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/disk.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/error.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/host.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/http.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/k8s.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/messaging.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/network.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/oci.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/os.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/process.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/rpc.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/server.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/source.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/thread.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/tls.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/url.yaml delete mode 100644 internal/otelschema/_testdata/model/registry/user-agent.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/android.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/browser.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/cloud.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/aws/ecs.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/aws/eks.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/aws/logs.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/gcp/cloud_run.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/gcp/gce.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/cloud_provider/heroku.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/container.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/deployment_environment.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/device.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/faas.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/host.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/k8s.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/os.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/process.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/service.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/service_experimental.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/telemetry.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/telemetry_experimental.yaml delete mode 100644 internal/otelschema/_testdata/model/resource/webengine.yaml delete mode 100644 internal/otelschema/_testdata/model/scope/exporter/exporter.yaml delete mode 100644 internal/otelschema/_testdata/model/session.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/aws/lambda.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/cloudevents.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/compatibility.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/database.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/exporter/exporter.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/faas.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/feature-flag.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/http.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/instrumentation/aws-sdk.yml delete mode 100644 internal/otelschema/_testdata/model/trace/instrumentation/graphql.yml delete mode 100644 internal/otelschema/_testdata/model/trace/messaging.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/rpc.yaml delete mode 100644 internal/otelschema/_testdata/model/trace/trace-exception.yaml delete mode 100644 internal/otelschema/_testdata/model/url.yaml delete mode 100644 internal/otelschema/_testdata/model/version.properties create mode 160000 internal/otelschema/_testdata/semantic-conventions delete mode 100644 internal/otelschema/list_test.go create mode 100644 internal/otelschema/registry_test.go diff --git a/.gitmodules b/.gitmodules new file mode 100644 index 00000000..7638688e --- /dev/null +++ b/.gitmodules @@ -0,0 +1,3 @@ +[submodule "internal/otelschema/_testdata/semantic-conventions"] + path = internal/otelschema/_testdata/semantic-conventions + url = https://github.com/open-telemetry/semantic-conventions.git diff --git a/internal/otelschema/_golden/all_attributes.yaml b/internal/otelschema/_golden/all_attributes.yaml deleted file mode 100644 index 5a077dc9..00000000 --- a/internal/otelschema/_golden/all_attributes.yaml +++ /dev/null @@ -1,850 +0,0 @@ -- Name: exception.type - Type: string -- Name: exception.message - Type: string -- Name: exception.stacktrace - Type: string -- Enum: - - datasource - - http - - pubsub - - timer - - other - Name: faas.trigger - Type: string -- Name: faas.invoked_name - Type: string -- Name: faas.invoked_provider - Type: string -- Name: faas.invoked_region - Type: string -- Name: peer.service - Type: string -- Name: enduser.id - Type: string -- Name: enduser.role - Type: string -- Name: enduser.scope - Type: string -- Name: event.name - Type: string -- Name: log.record.uid - Type: string -- Enum: - - stdout - - stderr - Name: log.iostream - Type: string -- Name: log.file.name - Type: string -- Name: log.file.path - Type: string -- Name: log.file.name_resolved - Type: string -- Name: log.file.path_resolved - Type: string -- Enum: - - active - - inactive - - background - - foreground - - terminate - Name: ios.state - Type: string -- Enum: - - created - - background - - foreground - Name: android.state - Type: string -- Enum: - - idle - - used - Name: state - Type: string -- Name: pool.name - Type: string -- Name: jvm.buffer.pool.name - Type: string -- Enum: - - heap - - non_heap - Name: jvm.memory.type - Type: string -- Name: jvm.memory.pool.name - Type: string -- Name: jvm.gc.name - Type: string -- Name: jvm.gc.action - Type: string -- Name: jvm.thread.daemon - Type: boolean -- Enum: - - new - - runnable - - blocked - - waiting - - timed_waiting - - terminated - Name: jvm.thread.state - Type: string -- Name: system.device - Type: string -- Name: system.cpu.state - Type: string -- Name: system.cpu.logical_number - Type: int -- Name: system.memory.state - Type: string -- Enum: - - used - - free - Name: system.paging.state - Type: string -- Enum: - - major - - minor - Name: system.paging.type - Type: string -- Enum: - - in - - out - Name: system.paging.direction - Type: string -- Enum: - - used - - free - - reserved - Name: system.filesystem.state - Type: string -- Name: system.filesystem.type - Type: string -- Name: system.filesystem.mode - Type: string -- Name: system.filesystem.mountpoint - Type: string -- Enum: - - close - - close_wait - - closing - - delete - - established - - fin_wait_1 - - fin_wait_2 - - last_ack - - listen - - syn_recv - - syn_sent - - time_wait - Name: system.network.state - Type: string -- Name: system.processes.status - Type: string -- Name: client.address - Type: string -- Name: client.port - Type: int -- Name: cloud.provider - Type: string -- Name: cloud.account.id - Type: string -- Name: cloud.region - Type: string -- Name: cloud.resource_id - Type: string -- Name: cloud.availability_zone - Type: string -- Name: cloud.platform - Type: string -- Name: code.function - Type: string -- Name: code.namespace - Type: string -- Name: code.filepath - Type: string -- Name: code.lineno - Type: int -- Name: code.column - Type: int -- Name: code.stacktrace - Type: string -- Name: container.name - Type: string -- Name: container.id - Type: string -- Name: container.runtime - Type: string -- Name: container.image.name - Type: string -- Name: container.image.tags - Type: string[] -- Name: container.image.id - Type: string -- Name: container.image.repo_digests - Type: string[] -- Name: container.command - Type: string -- Name: container.command_line - Type: string -- Name: container.command_args - Type: string[] -- Name: container.labels - Type: template[string] -- Name: db.cassandra.coordinator.dc - Type: string -- Name: db.cassandra.coordinator.id - Type: string -- Enum: - - all - - each_quorum - - quorum - - local_quorum - - one - - two - - three - - local_one - - any - - serial - - local_serial - Name: db.cassandra.consistency_level - Type: string -- Name: db.cassandra.idempotence - Type: boolean -- Name: db.cassandra.page_size - Type: int -- Name: db.cassandra.speculative_execution_count - Type: int -- Name: db.cassandra.table - Type: string -- Name: db.connection_string - Type: string -- Name: db.cosmosdb.client_id - Type: string -- Enum: - - gateway - - direct - Name: db.cosmosdb.connection_mode - Type: string -- Name: db.cosmosdb.container - Type: string -- Name: db.cosmosdb.operation_type - Type: string -- Name: db.cosmosdb.request_charge - Type: double -- Name: db.cosmosdb.request_content_length - Type: int -- Name: db.cosmosdb.status_code - Type: int -- Name: db.cosmosdb.sub_status_code - Type: int -- Name: db.elasticsearch.cluster.name - Type: string -- Name: db.elasticsearch.node.name - Type: string -- Name: db.elasticsearch.path_parts - Type: template[string] -- Name: db.jdbc.driver_classname - Type: string -- Name: db.mongodb.collection - Type: string -- Name: db.mssql.instance_name - Type: string -- Name: db.name - Type: string -- Name: db.operation - Type: string -- Name: db.redis.database_index - Type: int -- Name: db.sql.table - Type: string -- Name: db.statement - Type: string -- Name: db.system - Type: string -- Name: db.user - Type: string -- Name: http.method - Type: string -- Name: http.status_code - Type: int -- Name: http.scheme - Type: string -- Name: http.url - Type: string -- Name: http.target - Type: string -- Name: http.request_content_length - Type: int -- Name: http.response_content_length - Type: int -- Name: http.flavor - Type: string -- Name: http.user_agent - Type: string -- Name: net.sock.peer.name - Type: string -- Name: net.sock.peer.addr - Type: string -- Name: net.sock.peer.port - Type: int -- Name: net.peer.name - Type: string -- Name: net.peer.port - Type: int -- Name: net.host.name - Type: string -- Name: net.host.port - Type: int -- Name: net.sock.host.addr - Type: string -- Name: net.sock.host.port - Type: int -- Name: net.transport - Type: string -- Name: net.protocol.name - Type: string -- Name: net.protocol.version - Type: string -- Name: net.sock.family - Type: string -- Name: destination.address - Type: string -- Name: destination.port - Type: int -- Name: device.id - Type: string -- Name: device.manufacturer - Type: string -- Name: device.model.identifier - Type: string -- Name: device.model.name - Type: string -- Enum: - - read - - write - Name: disk.io.direction - Type: string -- Name: error.type - Type: string -- Name: host.id - Type: string -- Name: host.name - Type: string -- Name: host.type - Type: string -- Name: host.arch - Type: string -- Name: host.image.name - Type: string -- Name: host.image.id - Type: string -- Name: host.image.version - Type: string -- Name: host.ip - Type: string[] -- Name: host.mac - Type: string[] -- Name: host.cpu.vendor.id - Type: string -- Name: host.cpu.family - Type: string -- Name: host.cpu.model.id - Type: string -- Name: host.cpu.model.name - Type: string -- Name: host.cpu.stepping - Type: int -- Name: host.cpu.cache.l2.size - Type: int -- Name: http.request.body.size - Type: int -- Name: http.request.header - Type: template[string[]] -- Name: http.request.method - Type: string -- Name: http.request.method_original - Type: string -- Name: http.request.resend_count - Type: int -- Name: http.response.body.size - Type: int -- Name: http.response.header - Type: template[string[]] -- Name: http.response.status_code - Type: int -- Name: http.route - Type: string -- Name: k8s.cluster.name - Type: string -- Name: k8s.cluster.uid - Type: string -- Name: k8s.node.name - Type: string -- Name: k8s.node.uid - Type: string -- Name: k8s.namespace.name - Type: string -- Name: k8s.pod.uid - Type: string -- Name: k8s.pod.name - Type: string -- Name: k8s.container.name - Type: string -- Name: k8s.container.restart_count - Type: int -- Name: k8s.replicaset.uid - Type: string -- Name: k8s.replicaset.name - Type: string -- Name: k8s.deployment.uid - Type: string -- Name: k8s.deployment.name - Type: string -- Name: k8s.statefulset.uid - Type: string -- Name: k8s.statefulset.name - Type: string -- Name: k8s.daemonset.uid - Type: string -- Name: k8s.daemonset.name - Type: string -- Name: k8s.job.uid - Type: string -- Name: k8s.job.name - Type: string -- Name: k8s.cronjob.uid - Type: string -- Name: k8s.cronjob.name - Type: string -- Name: messaging.batch.message_count - Type: int -- Name: messaging.client_id - Type: string -- Name: messaging.destination.name - Type: string -- Name: messaging.destination.template - Type: string -- Name: messaging.destination.anonymous - Type: boolean -- Name: messaging.destination.temporary - Type: boolean -- Name: messaging.destination_publish.anonymous - Type: boolean -- Name: messaging.destination_publish.name - Type: string -- Name: messaging.kafka.consumer.group - Type: string -- Name: messaging.kafka.destination.partition - Type: int -- Name: messaging.kafka.message.key - Type: string -- Name: messaging.kafka.message.offset - Type: int -- Name: messaging.kafka.message.tombstone - Type: boolean -- Name: messaging.message.conversation_id - Type: string -- Name: messaging.message.envelope.size - Type: int -- Name: messaging.message.id - Type: string -- Name: messaging.message.body.size - Type: int -- Name: messaging.operation - Type: string -- Name: messaging.rabbitmq.destination.routing_key - Type: string -- Name: messaging.rocketmq.client_group - Type: string -- Enum: - - clustering - - broadcasting - Name: messaging.rocketmq.consumption_model - Type: string -- Name: messaging.rocketmq.message.delay_time_level - Type: int -- Name: messaging.rocketmq.message.delivery_timestamp - Type: int -- Name: messaging.rocketmq.message.group - Type: string -- Name: messaging.rocketmq.message.keys - Type: string[] -- Name: messaging.rocketmq.message.tag - Type: string -- Enum: - - normal - - fifo - - delay - - transaction - Name: messaging.rocketmq.message.type - Type: string -- Name: messaging.rocketmq.namespace - Type: string -- Name: messaging.gcp_pubsub.message.ordering_key - Type: string -- Name: messaging.system - Type: string -- Name: network.carrier.icc - Type: string -- Name: network.carrier.mcc - Type: string -- Name: network.carrier.mnc - Type: string -- Name: network.carrier.name - Type: string -- Name: network.connection.subtype - Type: string -- Name: network.connection.type - Type: string -- Name: network.local.address - Type: string -- Name: network.local.port - Type: int -- Name: network.peer.address - Type: string -- Name: network.peer.port - Type: int -- Name: network.protocol.name - Type: string -- Name: network.protocol.version - Type: string -- Name: network.transport - Type: string -- Name: network.type - Type: string -- Enum: - - transmit - - receive - Name: network.io.direction - Type: string -- Name: oci.manifest.digest - Type: string -- Name: os.type - Type: string -- Name: os.description - Type: string -- Name: os.name - Type: string -- Name: os.version - Type: string -- Name: os.build_id - Type: string -- Name: process.pid - Type: int -- Name: process.parent_pid - Type: int -- Name: process.executable.name - Type: string -- Name: process.executable.path - Type: string -- Name: process.command - Type: string -- Name: process.command_line - Type: string -- Name: process.command_args - Type: string[] -- Name: process.owner - Type: string -- Name: process.runtime.name - Type: string -- Name: process.runtime.version - Type: string -- Name: process.runtime.description - Type: string -- Enum: - - cancelled - - unknown - - invalid_argument - - deadline_exceeded - - not_found - - already_exists - - permission_denied - - resource_exhausted - - failed_precondition - - aborted - - out_of_range - - unimplemented - - internal - - unavailable - - data_loss - - unauthenticated - Name: rpc.connect_rpc.error_code - Type: string -- Name: rpc.connect_rpc.request.metadata - Type: template[string[]] -- Name: rpc.connect_rpc.response.metadata - Type: template[string[]] -- Enum: - - 0 - - 1 - - 2 - - 3 - - 4 - - 5 - - 6 - - 7 - - 8 - - 9 - - 10 - - 11 - - 12 - - 13 - - 14 - - 15 - - 16 - Name: rpc.grpc.status_code - Type: int -- Name: rpc.grpc.request.metadata - Type: template[string[]] -- Name: rpc.grpc.response.metadata - Type: template[string[]] -- Name: rpc.jsonrpc.error_code - Type: int -- Name: rpc.jsonrpc.error_message - Type: string -- Name: rpc.jsonrpc.request_id - Type: string -- Name: rpc.jsonrpc.version - Type: string -- Name: rpc.method - Type: string -- Name: rpc.service - Type: string -- Name: rpc.system - Type: string -- Name: server.address - Type: string -- Name: server.port - Type: int -- Name: source.address - Type: string -- Name: source.port - Type: int -- Name: thread.id - Type: int -- Name: thread.name - Type: string -- Name: tls.cipher - Type: string -- Name: tls.client.certificate - Type: string -- Name: tls.client.certificate_chain - Type: string[] -- Name: tls.client.hash.md5 - Type: string -- Name: tls.client.hash.sha1 - Type: string -- Name: tls.client.hash.sha256 - Type: string -- Name: tls.client.issuer - Type: string -- Name: tls.client.ja3 - Type: string -- Name: tls.client.not_after - Type: string -- Name: tls.client.not_before - Type: string -- Name: tls.client.server_name - Type: string -- Name: tls.client.subject - Type: string -- Name: tls.client.supported_ciphers - Type: string[] -- Name: tls.curve - Type: string -- Name: tls.established - Type: boolean -- Name: tls.next_protocol - Type: string -- Name: tls.protocol.name - Type: string -- Name: tls.protocol.version - Type: string -- Name: tls.resumed - Type: boolean -- Name: tls.server.certificate - Type: string -- Name: tls.server.certificate_chain - Type: string[] -- Name: tls.server.hash.md5 - Type: string -- Name: tls.server.hash.sha1 - Type: string -- Name: tls.server.hash.sha256 - Type: string -- Name: tls.server.issuer - Type: string -- Name: tls.server.ja3s - Type: string -- Name: tls.server.not_after - Type: string -- Name: tls.server.not_before - Type: string -- Name: tls.server.subject - Type: string -- Name: url.scheme - Type: string -- Name: url.full - Type: string -- Name: url.path - Type: string -- Name: url.query - Type: string -- Name: url.fragment - Type: string -- Name: user_agent.original - Type: string -- Name: android.os.api_level - Type: string -- Name: browser.brands - Type: string[] -- Name: browser.platform - Type: string -- Name: browser.mobile - Type: boolean -- Name: browser.language - Type: string -- Name: aws.ecs.container.arn - Type: string -- Name: aws.ecs.cluster.arn - Type: string -- Enum: - - ec2 - - fargate - Name: aws.ecs.launchtype - Type: string -- Name: aws.ecs.task.arn - Type: string -- Name: aws.ecs.task.family - Type: string -- Name: aws.ecs.task.revision - Type: string -- Name: aws.eks.cluster.arn - Type: string -- Name: aws.log.group.names - Type: string[] -- Name: aws.log.group.arns - Type: string[] -- Name: aws.log.stream.names - Type: string[] -- Name: aws.log.stream.arns - Type: string[] -- Name: gcp.cloud_run.job.execution - Type: string -- Name: gcp.cloud_run.job.task_index - Type: int -- Name: gcp.gce.instance.name - Type: string -- Name: gcp.gce.instance.hostname - Type: string -- Name: heroku.release.creation_timestamp - Type: string -- Name: heroku.release.commit - Type: string -- Name: heroku.app.id - Type: string -- Name: deployment.environment - Type: string -- Name: faas.name - Type: string -- Name: faas.version - Type: string -- Name: faas.instance - Type: string -- Name: faas.max_memory - Type: int -- Name: service.name - Type: string -- Name: service.version - Type: string -- Name: service.namespace - Type: string -- Name: service.instance.id - Type: string -- Name: telemetry.sdk.name - Type: string -- Name: telemetry.sdk.language - Type: string -- Name: telemetry.sdk.version - Type: string -- Name: telemetry.distro.name - Type: string -- Name: telemetry.distro.version - Type: string -- Name: webengine.name - Type: string -- Name: webengine.version - Type: string -- Name: webengine.description - Type: string -- Name: otel.scope.name - Type: string -- Name: otel.scope.version - Type: string -- Name: otel.library.name - Type: string -- Name: otel.library.version - Type: string -- Name: session.id - Type: string -- Name: session.previous_id - Type: string -- Name: aws.lambda.invoked_arn - Type: string -- Name: cloudevents.event_id - Type: string -- Name: cloudevents.event_source - Type: string -- Name: cloudevents.event_spec_version - Type: string -- Name: cloudevents.event_type - Type: string -- Name: cloudevents.event_subject - Type: string -- Enum: - - child_of - - follows_from - Name: opentracing.ref_type - Type: string -- Enum: - - OK - - ERROR - Name: otel.status_code - Type: string -- Name: otel.status_description - Type: string -- Name: faas.invocation_id - Type: string -- Name: faas.document.collection - Type: string -- Name: faas.document.operation - Type: string -- Name: faas.document.time - Type: string -- Name: faas.document.name - Type: string -- Name: faas.time - Type: string -- Name: faas.cron - Type: string -- Name: faas.coldstart - Type: boolean -- Name: feature_flag.key - Type: string -- Name: feature_flag.provider_name - Type: string -- Name: feature_flag.variant - Type: string -- Enum: - - SENT - - RECEIVED - Name: message.type - Type: string -- Name: message.id - Type: int -- Name: message.compressed_size - Type: int -- Name: message.uncompressed_size - Type: int -- Name: exception.escaped - Type: boolean diff --git a/internal/otelschema/_golden/registry.yaml b/internal/otelschema/_golden/registry.yaml new file mode 100644 index 00000000..cce2ccbd --- /dev/null +++ b/internal/otelschema/_golden/registry.yaml @@ -0,0 +1,2581 @@ +entries: + android.os.api_level: + brief: | + Uniquely identifies the framework API revision offered by a version (`os.version`) of the android operating system. More information can be found [here](https://developer.android.com/guide/topics/manifest/uses-sdk-element#ApiLevels). + column: String + examples: + - "33" + - "32" + type: string + android.state: + brief: | + This attribute represents the state the application has transitioned into at the occurrence of the event. + column: Enum8(0 = 'created', 1 = 'background', 2 = 'foreground') + enum: + - created + - background + - foreground + type: string + aws.ecs.cluster.arn: + brief: | + The ARN of an [ECS cluster](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/clusters.html). + column: String + examples: + - arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster + type: string + aws.ecs.container.arn: + brief: | + The Amazon Resource Name (ARN) of an [ECS container instance](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/ECS_instances.html). + column: String + examples: + - arn:aws:ecs:us-west-1:123456789123:container/32624152-9086-4f0e-acae-1a75b14fe4d9 + type: string + aws.ecs.launchtype: + brief: | + The [launch type](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/launch_types.html) for an ECS task. + column: Enum8(0 = 'ec2', 1 = 'fargate') + enum: + - ec2 + - fargate + type: string + aws.ecs.task.arn: + brief: | + The ARN of an [ECS task definition](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task_definitions.html). + column: String + examples: + - arn:aws:ecs:us-west-1:123456789123:task/10838bed-421f-43ef-870a-f43feacbbb5b + type: string + aws.ecs.task.family: + brief: | + The task definition family this task definition is a member of. + column: String + examples: + - opentelemetry-family + type: string + aws.ecs.task.revision: + brief: | + The revision for this task definition. + column: String + examples: + - "8" + - "26" + type: string + aws.eks.cluster.arn: + brief: | + The ARN of an EKS cluster. + column: String + examples: + - arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster + type: string + aws.lambda.invoked_arn: + brief: | + The full invoked ARN as provided on the `Context` passed to the function (`Lambda-Runtime-Invoked-Function-Arn` header on the `/runtime/invocation/next` applicable). + column: String + examples: + - arn:aws:lambda:us-east-1:123456:function:myfunction:myalias + type: string + aws.log.group.arns: + brief: | + The Amazon Resource Name(s) (ARN) of the AWS log group(s). + column: Array(String) + examples: + - arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:* + type: string[] + aws.log.group.names: + brief: | + The name(s) of the AWS log group(s) an application is writing to. + column: Array(String) + examples: + - /aws/lambda/my-function + - opentelemetry-service + type: string[] + aws.log.stream.arns: + brief: | + The ARN(s) of the AWS log stream(s). + column: Array(String) + examples: + - arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:log-stream:logs/main/10838bed-421f-43ef-870a-f43feacbbb5b + type: string[] + aws.log.stream.names: + brief: | + The name(s) of the AWS log stream(s) an application is writing to. + column: Array(String) + examples: + - logs/main/10838bed-421f-43ef-870a-f43feacbbb5b + type: string[] + browser.brands: + brief: Array of brand name and version separated by a space + column: Array(String) + examples: + - ' Not A;Brand 99' + - Chromium 99 + - Chrome 99 + type: string[] + browser.language: + brief: Preferred language of the user using the browser + column: String + examples: + - en + - en-US + - fr + - fr-FR + type: string + browser.mobile: + brief: A boolean that is true if the browser is running on a mobile device + column: Bool + type: boolean + browser.platform: + brief: The platform on which the browser is running + column: String + examples: + - Windows + - macOS + - Android + type: string + client.address: + brief: Client address - domain name if available without reverse DNS lookup; otherwise, + IP address or Unix domain socket name. + column: String + examples: + - client.example.com + - 10.1.2.80 + - /tmp/my.sock + type: string + client.port: + brief: Client port number. + column: UInt16 + examples: + - 65123 + type: int + cloud.account.id: + brief: | + The cloud account ID the resource is assigned to. + column: String + examples: + - "111111111111" + - opentelemetry + type: string + cloud.availability_zone: + brief: | + Cloud regions often have multiple, isolated locations known as zones to increase availability. Availability zone represents the zone where the resource is running. + column: String + examples: + - us-east-1c + type: string + cloud.platform: + brief: | + The cloud platform in use. + column: String + type: string + cloud.provider: + brief: | + Name of the cloud provider. + column: String + type: string + cloud.region: + brief: | + The geographical region the resource is running. + column: String + examples: + - us-central1 + - us-east-1 + type: string + cloud.resource_id: + brief: | + Cloud provider-specific native identifier of the monitored cloud resource (e.g. an [ARN](https://docs.aws.amazon.com/general/latest/gr/aws-arns-and-namespaces.html) on AWS, a [fully qualified resource ID](https://learn.microsoft.com/rest/api/resources/resources/get-by-id) on Azure, a [full resource name](https://cloud.google.com/apis/design/resource_names#full_resource_name) on GCP) + column: String + examples: + - arn:aws:lambda:REGION:ACCOUNT_ID:function:my-function + - //run.googleapis.com/projects/PROJECT_ID/locations/LOCATION_ID/services/SERVICE_ID + - /subscriptions//resourceGroups//providers/Microsoft.Web/sites//functions/ + type: string + cloudevents.event_id: + brief: | + The [event_id](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#id) uniquely identifies the event. + column: String + examples: + - 123e4567-e89b-12d3-a456-426614174000 + - "0001" + type: string + cloudevents.event_source: + brief: | + The [source](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#source-1) identifies the context in which an event happened. + column: String + examples: + - https://github.com/cloudevents + - /cloudevents/spec/pull/123 + - my-service + type: string + cloudevents.event_spec_version: + brief: | + The [version of the CloudEvents specification](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#specversion) which the event uses. + column: String + examples: + - "1.0" + type: string + cloudevents.event_subject: + brief: | + The [subject](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#subject) of the event in the context of the event producer (identified by source). + column: String + examples: + - mynewfile.jpg + type: string + cloudevents.event_type: + brief: | + The [event_type](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#type) contains a value describing the type of event related to the originating occurrence. + column: String + examples: + - com.github.pull_request.opened + - com.example.object.deleted.v2 + type: string + code.column: + brief: | + The column number in `code.filepath` best representing the operation. It SHOULD point within the code unit named in `code.function`. + column: Int64 + examples: + - 16 + type: int + code.filepath: + brief: | + The source code file name that identifies the code unit as uniquely as possible (preferably an absolute file path). + column: String + examples: + - /usr/local/MyApplication/content_root/app/index.php + type: string + code.function: + brief: | + The method or function name, or equivalent (usually rightmost part of the code unit's name). + column: String + examples: + - serveRequest + type: string + code.lineno: + brief: | + The line number in `code.filepath` best representing the operation. It SHOULD point within the code unit named in `code.function`. + column: Int64 + examples: + - 42 + type: int + code.namespace: + brief: | + The "namespace" within which `code.function` is defined. Usually the qualified class or module name, such that `code.namespace` + some separator + `code.function` form a unique identifier for the code unit. + column: String + examples: + - com.example.MyHttpService + type: string + code.stacktrace: + brief: | + A stacktrace as a string in the natural representation for the language runtime. The representation is to be determined and documented by each language SIG. + column: String + examples: + - at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n + at com.example.GenerateTrace.main(GenerateTrace.java:5) + type: string + container.command: + brief: | + The command used to run the container (i.e. the command name). + column: String + examples: + - otelcontribcol + type: string + container.command_args: + brief: | + All the command arguments (including the command/executable itself) run by the container. [2] + column: Array(String) + examples: + - otelcontribcol, --config, config.yaml + type: string[] + container.command_line: + brief: | + The full command run by the container as a single string representing the full command. [2] + column: String + examples: + - otelcontribcol --config config.yaml + type: string + container.id: + brief: | + Container ID. Usually a UUID, as for example used to [identify Docker containers](https://docs.docker.com/engine/reference/run/#container-identification). The UUID might be abbreviated. + column: String + examples: + - a3bf90e006b2 + type: string + container.image.id: + brief: | + Runtime specific image identifier. Usually a hash algorithm followed by a UUID. + column: String + examples: + - sha256:19c92d0a00d1b66d897bceaa7319bee0dd38a10a851c60bcec9474aa3f01e50f + type: string + container.image.name: + brief: | + Name of the image the container was built on. + column: String + examples: + - gcr.io/opentelemetry/operator + type: string + container.image.repo_digests: + brief: | + Repo digests of the container image as provided by the container runtime. + column: Array(String) + examples: + - example@sha256:afcc7f1ac1b49db317a7196c902e61c6c3c4607d63599ee1a82d702d249a0ccb + - internal.registry.example.com:5000/example@sha256:b69959407d21e8a062e0416bf13405bb2b71ed7a84dde4158ebafacfa06f5578 + type: string[] + container.image.tags: + brief: | + Container image tags. An example can be found in [Docker Image Inspect](https://docs.docker.com/engine/api/v1.43/#tag/Image/operation/ImageInspect). Should be only the `` section of the full name for example from `registry.example.com/my-org/my-image:`. + column: Array(String) + examples: + - v1.27.1 + - 3.5.7-0 + type: string[] + container.labels: + brief: | + Container labels, `` being the label name, the value being the label value. + column: String + examples: + - container.labels.app=nginx + type: template[string] + container.name: + brief: | + Container name used by container runtime. + column: String + examples: + - opentelemetry-autoconf + type: string + container.runtime: + brief: | + The container runtime managing this container. + column: String + examples: + - docker + - containerd + - rkt + type: string + db.cassandra.consistency_level: + brief: | + The consistency level of the query. Based on consistency values from [CQL](https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/dml/dmlConfigConsistency.html). + column: Enum8(0 = 'all', 1 = 'each_quorum', 2 = 'quorum', 3 = 'local_quorum', + 4 = 'one', 5 = 'two', 6 = 'three', 7 = 'local_one', 8 = 'any', 9 = 'serial', + 10 = 'local_serial') + enum: + - all + - each_quorum + - quorum + - local_quorum + - one + - two + - three + - local_one + - any + - serial + - local_serial + type: string + db.cassandra.coordinator.dc: + brief: | + The data center of the coordinating node for a query. + column: String + examples: + - us-west-2 + type: string + db.cassandra.coordinator.id: + brief: | + The ID of the coordinating node for a query. + column: String + examples: + - be13faa2-8574-4d71-926d-27f16cf8a7af + type: string + db.cassandra.idempotence: + brief: | + Whether or not the query is idempotent. + column: Bool + type: boolean + db.cassandra.page_size: + brief: | + The fetch size used for paging, i.e. how many rows will be returned at once. + column: Int64 + examples: + - 5000 + type: int + db.cassandra.speculative_execution_count: + brief: | + The number of times a query was speculatively executed. Not set or `0` if the query was not executed speculatively. + column: Int64 + examples: + - 0 + - 2 + type: int + db.cassandra.table: + brief: The name of the primary Cassandra table that the operation is acting upon, + including the keyspace name (if applicable). + column: String + examples: + - mytable + type: string + db.connection_string: + brief: | + The connection string used to connect to the database. It is recommended to remove embedded credentials. + column: String + examples: + - Server=(localdb)\v11.0;Integrated Security=true; + type: string + db.cosmosdb.client_id: + brief: Unique Cosmos client instance id. + column: String + examples: + - 3ba4827d-4422-483f-b59f-85b74211c11d + type: string + db.cosmosdb.connection_mode: + brief: Cosmos client connection mode. + column: Enum8(0 = 'gateway', 1 = 'direct') + enum: + - gateway + - direct + type: string + db.cosmosdb.container: + brief: Cosmos DB container name. + column: String + examples: + - anystring + type: string + db.cosmosdb.operation_type: + brief: CosmosDB Operation Type. + column: String + type: string + db.cosmosdb.request_charge: + brief: RU consumed for that operation + column: Float64 + examples: + - 46.18 + - 1 + type: double + db.cosmosdb.request_content_length: + brief: Request payload size in bytes + column: Int64 + type: int + db.cosmosdb.status_code: + brief: Cosmos DB status code. + column: Int64 + examples: + - 200 + - 201 + type: int + db.cosmosdb.sub_status_code: + brief: Cosmos DB sub status code. + column: Int64 + examples: + - 1000 + - 1002 + type: int + db.elasticsearch.cluster.name: + brief: | + Represents the identifier of an Elasticsearch cluster. + column: String + examples: + - e9106fc68e3044f0b1475b04bf4ffd5f + type: string + db.elasticsearch.node.name: + brief: | + Represents the human-readable identifier of the node/instance to which a request was routed. + column: String + examples: + - instance-0000000001 + type: string + db.elasticsearch.path_parts: + brief: | + A dynamic value in the url path. + column: String + examples: + - db.elasticsearch.path_parts.index=test-index + - db.elasticsearch.path_parts.doc_id=123 + type: template[string] + db.jdbc.driver_classname: + brief: | + The fully-qualified class name of the [Java Database Connectivity (JDBC)](https://docs.oracle.com/javase/8/docs/technotes/guides/jdbc/) driver used to connect. + column: String + examples: + - org.postgresql.Driver + - com.microsoft.sqlserver.jdbc.SQLServerDriver + type: string + db.mongodb.collection: + brief: | + The MongoDB collection being accessed within the database stated in `db.name`. + column: String + examples: + - customers + - products + type: string + db.mssql.instance_name: + brief: | + The Microsoft SQL Server [instance name](https://docs.microsoft.com/sql/connect/jdbc/building-the-connection-url?view=sql-server-ver15) connecting to. This name is used to determine the port of a named instance. + column: String + examples: + - MSSQLSERVER + type: string + db.name: + brief: | + This attribute is used to report the name of the database being accessed. For commands that switch the database, this should be set to the target database (even if the command fails). + column: String + examples: + - customers + - main + type: string + db.operation: + brief: | + The name of the operation being executed, e.g. the [MongoDB command name](https://docs.mongodb.com/manual/reference/command/#database-operations) such as `findAndModify`, or the SQL keyword. + column: String + examples: + - findAndModify + - HMSET + - SELECT + type: string + db.redis.database_index: + brief: | + The index of the database being accessed as used in the [`SELECT` command](https://redis.io/commands/select), provided as an integer. To be used instead of the generic `db.name` attribute. + column: Int64 + examples: + - 0 + - 1 + - 15 + type: int + db.sql.table: + brief: The name of the primary table that the operation is acting upon, including + the database name (if applicable). + column: String + examples: + - public.users + - customers + type: string + db.statement: + brief: | + The database statement being executed. + column: String + examples: + - SELECT * FROM wuser_table + - SET mykey "WuValue" + type: string + db.system: + brief: An identifier for the database management system (DBMS) product being used. + See below for a list of well-known identifiers. + column: String + type: string + db.user: + brief: | + Username for accessing the database. + column: String + examples: + - readonly_user + - reporting_user + type: string + deployment.environment: + brief: | + Name of the [deployment environment](https://wikipedia.org/wiki/Deployment_environment) (aka deployment tier). + column: String + examples: + - staging + - production + type: string + destination.address: + brief: Destination address - domain name if available without reverse DNS lookup; + otherwise, IP address or Unix domain socket name. + column: String + examples: + - destination.example.com + - 10.1.2.80 + - /tmp/my.sock + type: string + destination.port: + brief: Destination port number + column: UInt16 + examples: + - 3389 + - 2888 + type: int + device.id: + brief: | + A unique identifier representing the device + column: String + examples: + - 2ab2916d-a51f-4ac8-80ee-45ac31a28092 + type: string + device.manufacturer: + brief: | + The name of the device manufacturer + column: String + examples: + - Apple + - Samsung + type: string + device.model.identifier: + brief: | + The model identifier for the device + column: String + examples: + - iPhone3,4 + - SM-G920F + type: string + device.model.name: + brief: | + The marketing name for the device model + column: String + examples: + - iPhone 6s Plus + - Samsung Galaxy S6 + type: string + disk.io.direction: + brief: The disk IO operation direction. + column: Enum8(0 = 'read', 1 = 'write') + enum: + - read + - write + type: string + enduser.id: + brief: | + Username or client_id extracted from the access token or [Authorization](https://tools.ietf.org/html/rfc7235#section-4.2) header in the inbound request from outside the system. + column: String + examples: + - username + type: string + enduser.role: + brief: Actual/assumed role the client is making the request under extracted from + token or application security context. + column: String + examples: + - admin + type: string + enduser.scope: + brief: | + Scopes or granted authorities the client currently possesses extracted from token or application security context. The value would come from the scope associated with an [OAuth 2.0 Access Token](https://tools.ietf.org/html/rfc6749#section-3.3) or an attribute value in a [SAML 2.0 Assertion](http://docs.oasis-open.org/security/saml/Post2.0/sstc-saml-tech-overview-2.0.html). + column: String + examples: + - read:message, write:files + type: string + error.type: + brief: | + Describes a class of error the operation ended with. + column: String + examples: + - timeout + - java.net.UnknownHostException + - server_certificate_invalid + - "500" + type: string + event.name: + brief: | + Identifies the class / type of event. + column: String + examples: + - browser.mouse.click + - device.app.lifecycle + type: string + exception.escaped: + brief: | + SHOULD be set to true if the exception event is recorded at a point where it is known that the exception is escaping the scope of the span. + column: Bool + type: boolean + exception.message: + brief: The exception message. + column: String + examples: + - Division by zero + - Can't convert 'int' object to str implicitly + type: string + exception.stacktrace: + brief: | + A stacktrace as a string in the natural representation for the language runtime. The representation is to be determined and documented by each language SIG. + column: String + examples: + - 'Exception in thread "main" java.lang.RuntimeException: Test exception\n at + com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n + at com.example.GenerateTrace.main(GenerateTrace.java:5)' + type: string + exception.type: + brief: | + The type of the exception (its fully-qualified class name, if applicable). The dynamic type of the exception should be preferred over the static type in languages that support it. + column: String + examples: + - java.net.ConnectException + - OSError + type: string + faas.coldstart: + brief: | + A boolean that is true if the serverless function is executed for the first time (aka cold-start). + column: Bool + type: boolean + faas.cron: + brief: | + A string containing the schedule period as [Cron Expression](https://docs.oracle.com/cd/E12058_01/doc/doc.1014/e12030/cron_expressions.htm). + column: String + examples: + - 0/5 * * * ? * + type: string + faas.document.collection: + brief: | + The name of the source on which the triggering operation was performed. For example, in Cloud Storage or S3 corresponds to the bucket name, and in Cosmos DB to the database name. + column: String + examples: + - myBucketName + - myDbName + type: string + faas.document.name: + brief: | + The document name/table subjected to the operation. For example, in Cloud Storage or S3 is the name of the file, and in Cosmos DB the table name. + column: String + examples: + - myFile.txt + - myTableName + type: string + faas.document.operation: + brief: Describes the type of the operation that was performed on the data. + column: String + type: string + faas.document.time: + brief: | + A string containing the time when the data was accessed in the [ISO 8601](https://www.iso.org/iso-8601-date-and-time-format.html) format expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). + column: String + examples: + - "2020-01-23T13:47:06Z" + type: string + faas.instance: + brief: | + The execution environment ID as a string, that will be potentially reused for other invocations to the same function/function version. + column: String + examples: + - 2021/06/28/[$LATEST]2f399eb14537447da05ab2a2e39309de + type: string + faas.invocation_id: + brief: The invocation ID of the current function invocation. + column: String + examples: + - af9d5aa4-a685-4c5f-a22b-444f80b3cc28 + type: string + faas.invoked_name: + brief: | + The name of the invoked function. + column: String + examples: + - my-function + type: string + faas.invoked_provider: + brief: | + The cloud provider of the invoked function. + column: String + type: string + faas.invoked_region: + brief: | + The cloud region of the invoked function. + column: String + examples: + - eu-central-1 + type: string + faas.max_memory: + brief: | + The amount of memory available to the serverless function converted to Bytes. + column: Int64 + examples: + - 134217728 + type: int + faas.name: + brief: | + The name of the single function that this runtime instance executes. + column: String + examples: + - my-function + - myazurefunctionapp/some-function-name + type: string + faas.time: + brief: | + A string containing the function invocation time in the [ISO 8601](https://www.iso.org/iso-8601-date-and-time-format.html) format expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). + column: String + examples: + - "2020-01-23T13:47:06Z" + type: string + faas.trigger: + brief: Type of the trigger which caused this function invocation. + column: Enum8(0 = 'datasource', 1 = 'http', 2 = 'pubsub', 3 = 'timer', 4 = 'other') + enum: + - datasource + - http + - pubsub + - timer + - other + type: string + faas.version: + brief: The immutable version of the function being executed. + column: String + examples: + - "26" + - pinkfroid-00002 + type: string + feature_flag.key: + brief: The unique identifier of the feature flag. + column: String + examples: + - logo-color + type: string + feature_flag.provider_name: + brief: The name of the service provider that performs the flag evaluation. + column: String + examples: + - Flag Manager + type: string + feature_flag.variant: + brief: | + SHOULD be a semantic identifier for a value. If one is unavailable, a stringified version of the value can be used. + column: String + examples: + - red + - "true" + - "on" + type: string + gcp.cloud_run.job.execution: + brief: | + The name of the Cloud Run [execution](https://cloud.google.com/run/docs/managing/job-executions) being run for the Job, as set by the [`CLOUD_RUN_EXECUTION`](https://cloud.google.com/run/docs/container-contract#jobs-env-vars) environment variable. + column: String + examples: + - job-name-xxxx + - sample-job-mdw84 + type: string + gcp.cloud_run.job.task_index: + brief: | + The index for a task within an execution as provided by the [`CLOUD_RUN_TASK_INDEX`](https://cloud.google.com/run/docs/container-contract#jobs-env-vars) environment variable. + column: Int64 + examples: + - 0 + - 1 + type: int + gcp.gce.instance.hostname: + brief: | + The hostname of a GCE instance. This is the full value of the default or [custom hostname](https://cloud.google.com/compute/docs/instances/custom-hostname-vm). + column: String + examples: + - my-host1234.example.com + - sample-vm.us-west1-b.c.my-project.internal + type: string + gcp.gce.instance.name: + brief: | + The instance name of a GCE instance. This is the value provided by `host.name`, the visible name of the instance in the Cloud Console UI, and the prefix for the default hostname of the instance as defined by the [default internal DNS name](https://cloud.google.com/compute/docs/internal-dns#instance-fully-qualified-domain-names). + column: String + examples: + - instance-1 + - my-vm-name + type: string + heroku.app.id: + brief: | + Unique identifier for the application + column: String + examples: + - 2daa2797-e42b-4624-9322-ec3f968df4da + type: string + heroku.release.commit: + brief: | + Commit hash for the current release + column: String + examples: + - e6134959463efd8966b20e75b913cafe3f5ec + type: string + heroku.release.creation_timestamp: + brief: | + Time and date the release was created + column: String + examples: + - "2022-10-23T18:00:42Z" + type: string + host.arch: + brief: | + The CPU architecture the host system is running on. + column: String + type: string + host.cpu.cache.l2.size: + brief: | + The amount of level 2 memory cache available to the processor (in Bytes). + column: Int64 + examples: + - 12288000 + type: int + host.cpu.family: + brief: | + Family or generation of the CPU. + column: String + examples: + - "6" + - PA-RISC 1.1e + type: string + host.cpu.model.id: + brief: | + Model identifier. It provides more granular information about the CPU, distinguishing it from other CPUs within the same family. + column: String + examples: + - "6" + - 9000/778/B180L + type: string + host.cpu.model.name: + brief: | + Model designation of the processor. + column: String + examples: + - 11th Gen Intel(R) Core(TM) i7-1185G7 @ 3.00GHz + type: string + host.cpu.stepping: + brief: | + Stepping or core revisions. + column: Int64 + examples: + - 1 + type: int + host.cpu.vendor.id: + brief: | + Processor manufacturer identifier. A maximum 12-character string. + column: String + examples: + - GenuineIntel + type: string + host.id: + brief: | + Unique host ID. For Cloud, this must be the instance_id assigned by the cloud provider. For non-containerized systems, this should be the `machine-id`. See the table below for the sources to use to determine the `machine-id` based on operating system. + column: String + examples: + - fdbf79e8af94cb7f9e8df36789187052 + type: string + host.image.id: + brief: | + VM image ID or host OS image ID. For Cloud, this value is from the provider. + column: String + examples: + - ami-07b06b442921831e5 + type: string + host.image.name: + brief: | + Name of the VM image or OS install the host was instantiated from. + column: String + examples: + - infra-ami-eks-worker-node-7d4ec78312 + - CentOS-8-x86_64-1905 + type: string + host.image.version: + brief: | + The version string of the VM image or host OS as defined in [Version Attributes](/docs/resource/README.md#version-attributes). + column: String + examples: + - "0.1" + type: string + host.ip: + brief: | + Available IP addresses of the host, excluding loopback interfaces. + column: Array(String) + examples: + - 192.168.1.140 + - fe80::abc2:4a28:737a:609e + type: string[] + host.mac: + brief: | + Available MAC addresses of the host, excluding loopback interfaces. + column: Array(String) + examples: + - AC-DE-48-23-45-67 + - AC-DE-48-23-45-67-01-9F + type: string[] + host.name: + brief: | + Name of the host. On Unix systems, it may contain what the hostname command returns, or the fully qualified hostname, or another name specified by the user. + column: String + examples: + - opentelemetry-test + type: string + host.type: + brief: | + Type of host. For Cloud, this must be the machine type. + column: String + examples: + - n1-standard-1 + type: string + http.flavor: + brief: Deprecated, use `network.protocol.name` instead. + column: String + type: string + http.method: + brief: Deprecated, use `http.request.method` instead. + column: String + examples: + - GET + - POST + - HEAD + type: string + http.request.body.size: + brief: | + The size of the request payload body in bytes. This is the number of bytes transferred excluding headers and is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) header. For requests using transport encoding, this should be the compressed size. + column: Int64 + examples: + - 3495 + type: int + http.request.header: + brief: | + HTTP request headers, `` being the normalized HTTP Header name (lowercase), the value being the header values. + column: Array(String) + examples: + - http.request.header.content-type=["application/json"] + - http.request.header.x-forwarded-for=["1.2.3.4", "1.2.3.5"] + type: template[string[]] + http.request.method: + brief: HTTP request method. + column: String + examples: + - GET + - POST + - HEAD + type: string + http.request.method_original: + brief: Original HTTP method sent by the client in the request line. + column: String + examples: + - GeT + - ACL + - foo + type: string + http.request.resend_count: + brief: | + The ordinal number of request resending attempt (for any reason, including redirects). + column: Int64 + examples: + - 3 + type: int + http.request_content_length: + brief: Deprecated, use `http.request.header.content-length` instead. + column: Int64 + examples: + - 3495 + type: int + http.response.body.size: + brief: | + The size of the response payload body in bytes. This is the number of bytes transferred excluding headers and is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) header. For requests using transport encoding, this should be the compressed size. + column: Int64 + examples: + - 3495 + type: int + http.response.header: + brief: | + HTTP response headers, `` being the normalized HTTP Header name (lowercase), the value being the header values. + column: Array(String) + examples: + - http.response.header.content-type=["application/json"] + - http.response.header.my-custom-header=["abc", "def"] + type: template[string[]] + http.response.status_code: + brief: '[HTTP response status code](https://tools.ietf.org/html/rfc7231#section-6).' + column: Int64 + examples: + - 200 + type: int + http.response_content_length: + brief: Deprecated, use `http.response.header.content-length` instead. + column: Int64 + examples: + - 3495 + type: int + http.route: + brief: | + The matched route, that is, the path template in the format used by the respective server framework. + column: String + examples: + - /users/:userID? + - '{controller}/{action}/{id?}' + type: string + http.scheme: + brief: Deprecated, use `url.scheme` instead. + column: String + examples: + - http + - https + type: string + http.status_code: + brief: Deprecated, use `http.response.status_code` instead. + column: Int64 + examples: + - 200 + type: int + http.target: + brief: Deprecated, use `url.path` and `url.query` instead. + column: String + examples: + - /search?q=OpenTelemetry#SemConv + type: string + http.url: + brief: Deprecated, use `url.full` instead. + column: String + examples: + - https://www.foo.bar/search?q=OpenTelemetry#SemConv + type: string + http.user_agent: + brief: Deprecated, use `user_agent.original` instead. + column: String + examples: + - CERN-LineMode/2.15 libwww/2.17b3 + - Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 + (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1 + type: string + ios.state: + brief: | + This attribute represents the state the application has transitioned into at the occurrence of the event. + column: Enum8(0 = 'active', 1 = 'inactive', 2 = 'background', 3 = 'foreground', + 4 = 'terminate') + enum: + - active + - inactive + - background + - foreground + - terminate + type: string + jvm.buffer.pool.name: + brief: Name of the buffer pool. + column: String + examples: + - mapped + - direct + type: string + jvm.gc.action: + brief: Name of the garbage collector action. + column: String + examples: + - end of minor GC + - end of major GC + type: string + jvm.gc.name: + brief: Name of the garbage collector. + column: String + examples: + - G1 Young Generation + - G1 Old Generation + type: string + jvm.memory.pool.name: + brief: Name of the memory pool. + column: String + examples: + - G1 Old Gen + - G1 Eden space + - G1 Survivor Space + type: string + jvm.memory.type: + brief: The type of memory. + column: Enum8(0 = 'heap', 1 = 'non_heap') + enum: + - heap + - non_heap + type: string + jvm.thread.daemon: + brief: Whether the thread is daemon or not. + column: Bool + type: boolean + jvm.thread.state: + brief: State of the thread. + column: Enum8(0 = 'new', 1 = 'runnable', 2 = 'blocked', 3 = 'waiting', 4 = 'timed_waiting', + 5 = 'terminated') + enum: + - new + - runnable + - blocked + - waiting + - timed_waiting + - terminated + type: string + k8s.cluster.name: + brief: | + The name of the cluster. + column: String + examples: + - opentelemetry-cluster + type: string + k8s.cluster.uid: + brief: | + A pseudo-ID for the cluster, set to the UID of the `kube-system` namespace. + column: UUID + examples: + - 218fc5a9-a5f1-4b54-aa05-46717d0ab26d + type: string + k8s.container.name: + brief: | + The name of the Container from Pod specification, must be unique within a Pod. Container runtime usually uses different globally unique name (`container.name`). + column: String + examples: + - redis + type: string + k8s.container.restart_count: + brief: | + Number of times the container was restarted. This attribute can be used to identify a particular container (running or stopped) within a container spec. + column: Int64 + examples: + - 0 + - 2 + type: int + k8s.cronjob.name: + brief: | + The name of the CronJob. + column: String + examples: + - opentelemetry + type: string + k8s.cronjob.uid: + brief: | + The UID of the CronJob. + column: UUID + examples: + - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + type: string + k8s.daemonset.name: + brief: | + The name of the DaemonSet. + column: String + examples: + - opentelemetry + type: string + k8s.daemonset.uid: + brief: | + The UID of the DaemonSet. + column: UUID + examples: + - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + type: string + k8s.deployment.name: + brief: | + The name of the Deployment. + column: String + examples: + - opentelemetry + type: string + k8s.deployment.uid: + brief: | + The UID of the Deployment. + column: UUID + examples: + - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + type: string + k8s.job.name: + brief: | + The name of the Job. + column: String + examples: + - opentelemetry + type: string + k8s.job.uid: + brief: | + The UID of the Job. + column: UUID + examples: + - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + type: string + k8s.namespace.name: + brief: | + The name of the namespace that the pod is running in. + column: String + examples: + - default + type: string + k8s.node.name: + brief: | + The name of the Node. + column: String + examples: + - node-1 + type: string + k8s.node.uid: + brief: | + The UID of the Node. + column: UUID + examples: + - 1eb3a0c6-0477-4080-a9cb-0cb7db65c6a2 + type: string + k8s.pod.name: + brief: | + The name of the Pod. + column: String + examples: + - opentelemetry-pod-autoconf + type: string + k8s.pod.uid: + brief: | + The UID of the Pod. + column: UUID + examples: + - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + type: string + k8s.replicaset.name: + brief: | + The name of the ReplicaSet. + column: String + examples: + - opentelemetry + type: string + k8s.replicaset.uid: + brief: | + The UID of the ReplicaSet. + column: UUID + examples: + - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + type: string + k8s.statefulset.name: + brief: | + The name of the StatefulSet. + column: String + examples: + - opentelemetry + type: string + k8s.statefulset.uid: + brief: | + The UID of the StatefulSet. + column: UUID + examples: + - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + type: string + log.file.name: + brief: | + The basename of the file. + column: String + examples: + - audit.log + type: string + log.file.name_resolved: + brief: | + The basename of the file, with symlinks resolved. + column: String + examples: + - uuid.log + type: string + log.file.path: + brief: | + The full path to the file. + column: String + examples: + - /var/log/mysql/audit.log + type: string + log.file.path_resolved: + brief: | + The full path to the file, with symlinks resolved. + column: String + examples: + - /var/lib/docker/uuid.log + type: string + log.iostream: + brief: | + The stream associated with the log. See below for a list of well-known values. + column: Enum8(0 = 'stdout', 1 = 'stderr') + enum: + - stdout + - stderr + type: string + log.record.uid: + brief: | + A unique identifier for the Log Record. + column: String + examples: + - 01ARZ3NDEKTSV4RRFFQ69G5FAV + type: string + message.compressed_size: + brief: Compressed size of the message in bytes. + column: Int64 + type: int + message.id: + brief: MUST be calculated as two different counters starting from `1` one for + sent messages and one for received message. + column: Int64 + type: int + message.type: + brief: Whether this is a received or sent message. + column: Enum8(0 = 'SENT', 1 = 'RECEIVED') + enum: + - SENT + - RECEIVED + type: string + message.uncompressed_size: + brief: Uncompressed size of the message in bytes. + column: Int64 + type: int + messaging.batch.message_count: + brief: The number of messages sent, received, or processed in the scope of the + batching operation. + column: Int64 + examples: + - 0 + - 1 + - 2 + type: int + messaging.client_id: + brief: | + A unique identifier for the client that consumes or produces a message. + column: String + examples: + - client-5 + - myhost@8742@s8083jm + type: string + messaging.destination.anonymous: + brief: A boolean that is true if the message destination is anonymous (could be + unnamed or have auto-generated name). + column: Bool + type: boolean + messaging.destination.name: + brief: The message destination name + column: String + examples: + - MyQueue + - MyTopic + type: string + messaging.destination.template: + brief: Low cardinality representation of the messaging destination name + column: String + examples: + - /customers/{customerId} + type: string + messaging.destination.temporary: + brief: A boolean that is true if the message destination is temporary and might + not exist anymore after messages are processed. + column: Bool + type: boolean + messaging.destination_publish.anonymous: + brief: A boolean that is true if the publish message destination is anonymous + (could be unnamed or have auto-generated name). + column: Bool + type: boolean + messaging.destination_publish.name: + brief: The name of the original destination the message was published to + column: String + examples: + - MyQueue + - MyTopic + type: string + messaging.gcp_pubsub.message.ordering_key: + brief: | + The ordering key for a given message. If the attribute is not present, the message does not have an ordering key. + column: String + examples: + - ordering_key + type: string + messaging.kafka.consumer.group: + brief: | + Name of the Kafka Consumer Group that is handling the message. Only applies to consumers, not producers. + column: String + examples: + - my-group + type: string + messaging.kafka.destination.partition: + brief: | + Partition the message is sent to. + column: Int64 + examples: + - 2 + type: int + messaging.kafka.message.key: + brief: | + Message keys in Kafka are used for grouping alike messages to ensure they're processed on the same partition. They differ from `messaging.message.id` in that they're not unique. If the key is `null`, the attribute MUST NOT be set. + column: String + examples: + - myKey + type: string + messaging.kafka.message.offset: + brief: | + The offset of a record in the corresponding Kafka partition. + column: Int64 + examples: + - 42 + type: int + messaging.kafka.message.tombstone: + brief: A boolean that is true if the message is a tombstone. + column: Bool + type: boolean + messaging.message.body.size: + brief: | + The size of the message body in bytes. + column: Int64 + examples: + - 1439 + type: int + messaging.message.conversation_id: + brief: | + The conversation ID identifying the conversation to which the message belongs, represented as a string. Sometimes called "Correlation ID". + column: String + examples: + - MyConversationId + type: string + messaging.message.envelope.size: + brief: | + The size of the message body and metadata in bytes. + column: Int64 + examples: + - 2738 + type: int + messaging.message.id: + brief: A value used by the messaging system as an identifier for the message, + represented as a string. + column: String + examples: + - 452a7c7c7c7048c2f887f61572b18fc2 + type: string + messaging.operation: + brief: | + A string identifying the kind of messaging operation. + column: String + type: string + messaging.rabbitmq.destination.routing_key: + brief: | + RabbitMQ message routing key. + column: String + examples: + - myKey + type: string + messaging.rocketmq.client_group: + brief: | + Name of the RocketMQ producer/consumer group that is handling the message. The client type is identified by the SpanKind. + column: String + examples: + - myConsumerGroup + type: string + messaging.rocketmq.consumption_model: + brief: | + Model of message consumption. This only applies to consumer spans. + column: Enum8(0 = 'clustering', 1 = 'broadcasting') + enum: + - clustering + - broadcasting + type: string + messaging.rocketmq.message.delay_time_level: + brief: | + The delay time level for delay message, which determines the message delay time. + column: Int64 + examples: + - 3 + type: int + messaging.rocketmq.message.delivery_timestamp: + brief: | + The timestamp in milliseconds that the delay message is expected to be delivered to consumer. + column: Int64 + examples: + - 1665987217045 + type: int + messaging.rocketmq.message.group: + brief: | + It is essential for FIFO message. Messages that belong to the same message group are always processed one by one within the same consumer group. + column: String + examples: + - myMessageGroup + type: string + messaging.rocketmq.message.keys: + brief: | + Key(s) of message, another way to mark message besides message id. + column: Array(String) + examples: + - keyA + - keyB + type: string[] + messaging.rocketmq.message.tag: + brief: | + The secondary classifier of message besides topic. + column: String + examples: + - tagA + type: string + messaging.rocketmq.message.type: + brief: | + Type of message. + column: Enum8(0 = 'normal', 1 = 'fifo', 2 = 'delay', 3 = 'transaction') + enum: + - normal + - fifo + - delay + - transaction + type: string + messaging.rocketmq.namespace: + brief: | + Namespace of RocketMQ resources, resources in different namespaces are individual. + column: String + examples: + - myNamespace + type: string + messaging.system: + brief: | + An identifier for the messaging system being used. See below for a list of well-known identifiers. + column: String + type: string + net.host.name: + brief: Deprecated, use `server.address`. + column: String + examples: + - example.com + type: string + net.host.port: + brief: Deprecated, use `server.port`. + column: UInt16 + examples: + - 8080 + type: int + net.peer.name: + brief: Deprecated, use `server.address` on client spans and `client.address` on + server spans. + column: String + examples: + - example.com + type: string + net.peer.port: + brief: Deprecated, use `server.port` on client spans and `client.port` on server + spans. + column: UInt16 + examples: + - 8080 + type: int + net.protocol.name: + brief: Deprecated, use `network.protocol.name`. + column: String + examples: + - amqp + - http + - mqtt + type: string + net.protocol.version: + brief: Deprecated, use `network.protocol.version`. + column: String + examples: + - 3.1.1 + type: string + net.sock.family: + brief: Deprecated, use `network.transport` and `network.type`. + column: String + type: string + net.sock.host.addr: + brief: Deprecated, use `network.local.address`. + column: String + examples: + - /var/my.sock + type: string + net.sock.host.port: + brief: Deprecated, use `network.local.port`. + column: UInt16 + examples: + - 8080 + type: int + net.sock.peer.addr: + brief: Deprecated, use `network.peer.address`. + column: String + examples: + - 192.168.0.1 + type: string + net.sock.peer.name: + brief: Deprecated, no replacement at this time. + column: String + examples: + - /var/my.sock + type: string + net.sock.peer.port: + brief: Deprecated, use `network.peer.port`. + column: UInt16 + examples: + - 65531 + type: int + net.transport: + brief: Deprecated, use `network.transport`. + column: String + type: string + network.carrier.icc: + brief: The ISO 3166-1 alpha-2 2-character country code associated with the mobile + carrier network. + column: String + examples: + - DE + type: string + network.carrier.mcc: + brief: The mobile carrier country code. + column: String + examples: + - "310" + type: string + network.carrier.mnc: + brief: The mobile carrier network code. + column: String + examples: + - "001" + type: string + network.carrier.name: + brief: The name of the mobile carrier. + column: String + examples: + - sprint + type: string + network.connection.subtype: + brief: This describes more details regarding the connection.type. It may be the + type of cell technology connection, but it could be used for describing details + about a wifi connection. + column: String + examples: + - LTE + type: string + network.connection.type: + brief: The internet connection type. + column: String + examples: + - wifi + type: string + network.io.direction: + brief: The network IO operation direction. + column: Enum8(0 = 'transmit', 1 = 'receive') + enum: + - transmit + - receive + type: string + network.local.address: + brief: Local address of the network connection - IP address or Unix domain socket + name. + column: String + examples: + - 10.1.2.80 + - /tmp/my.sock + type: string + network.local.port: + brief: Local port number of the network connection. + column: UInt16 + examples: + - 65123 + type: int + network.peer.address: + brief: Peer address of the network connection - IP address or Unix domain socket + name. + column: String + examples: + - 10.1.2.80 + - /tmp/my.sock + type: string + network.peer.port: + brief: Peer port number of the network connection. + column: UInt16 + examples: + - 65123 + type: int + network.protocol.name: + brief: '[OSI application layer](https://osi-model.com/application-layer/) or non-OSI + equivalent.' + column: String + examples: + - amqp + - http + - mqtt + type: string + network.protocol.version: + brief: Version of the protocol specified in `network.protocol.name`. + column: String + examples: + - 3.1.1 + type: string + network.transport: + brief: | + [OSI transport layer](https://osi-model.com/transport-layer/) or [inter-process communication method](https://wikipedia.org/wiki/Inter-process_communication). + column: String + examples: + - tcp + - udp + type: string + network.type: + brief: '[OSI network layer](https://osi-model.com/network-layer/) or non-OSI equivalent.' + column: String + examples: + - ipv4 + - ipv6 + type: string + oci.manifest.digest: + brief: | + The digest of the OCI image manifest. For container images specifically is the digest by which the container image is known. + column: String + examples: + - sha256:e4ca62c0d62f3e886e684806dfe9d4e0cda60d54986898173c1083856cfda0f4 + type: string + opentracing.ref_type: + brief: Parent-child Reference type + column: Enum8(0 = 'child_of', 1 = 'follows_from') + enum: + - child_of + - follows_from + type: string + os.build_id: + brief: Unique identifier for a particular build or compilation of the operating + system. + column: String + examples: + - TQ3C.230805.001.B2 + - "20E247" + - "22621" + type: string + os.description: + brief: | + Human readable (not intended to be parsed) OS version information, like e.g. reported by `ver` or `lsb_release -a` commands. + column: String + examples: + - Microsoft Windows [Version 10.0.18363.778] + - Ubuntu 18.04.1 LTS + type: string + os.name: + brief: Human readable operating system name. + column: String + examples: + - iOS + - Android + - Ubuntu + type: string + os.type: + brief: | + The operating system type. + column: String + type: string + os.version: + brief: | + The version string of the operating system as defined in [Version Attributes](/docs/resource/README.md#version-attributes). + column: String + examples: + - 14.2.1 + - 18.04.1 + type: string + otel.library.name: + brief: Deprecated, use the `otel.scope.name` attribute. + column: String + examples: + - io.opentelemetry.contrib.mongodb + type: string + otel.library.version: + brief: Deprecated, use the `otel.scope.version` attribute. + column: String + examples: + - 1.0.0 + type: string + otel.scope.name: + brief: The name of the instrumentation scope - (`InstrumentationScope.Name` in + OTLP). + column: String + examples: + - io.opentelemetry.contrib.mongodb + type: string + otel.scope.version: + brief: The version of the instrumentation scope - (`InstrumentationScope.Version` + in OTLP). + column: String + examples: + - 1.0.0 + type: string + otel.status_code: + brief: Name of the code, either "OK" or "ERROR". MUST NOT be set if the status + code is UNSET. + column: Enum8(0 = 'OK', 1 = 'ERROR') + enum: + - OK + - ERROR + type: string + otel.status_description: + brief: Description of the Status if it has a value, otherwise not set. + column: String + examples: + - resource not found + type: string + peer.service: + brief: | + The [`service.name`](/docs/resource/README.md#service) of the remote service. SHOULD be equal to the actual `service.name` resource attribute of the remote service if any. + column: String + examples: + - AuthTokenCache + type: string + pool.name: + brief: | + The name of the connection pool; unique within the instrumented application. In case the connection pool implementation doesn't provide a name, then the [db.connection_string](/docs/database/database-spans.md#connection-level-attributes) should be used + column: String + examples: + - myDataSource + type: string + process.command: + brief: | + The command used to launch the process (i.e. the command name). On Linux based systems, can be set to the zeroth string in `proc/[pid]/cmdline`. On Windows, can be set to the first parameter extracted from `GetCommandLineW`. + column: String + examples: + - cmd/otelcol + type: string + process.command_args: + brief: | + All the command arguments (including the command/executable itself) as received by the process. On Linux-based systems (and some other Unixoid systems supporting procfs), can be set according to the list of null-delimited strings extracted from `proc/[pid]/cmdline`. For libc-based executables, this would be the full argv vector passed to `main`. + column: Array(String) + examples: + - cmd/otecol + - --config=config.yaml + type: string[] + process.command_line: + brief: | + The full command used to launch the process as a single string representing the full command. On Windows, can be set to the result of `GetCommandLineW`. Do not set this if you have to assemble it just for monitoring; use `process.command_args` instead. + column: String + examples: + - C:\cmd\otecol --config="my directory\config.yaml" + type: string + process.executable.name: + brief: | + The name of the process executable. On Linux based systems, can be set to the `Name` in `proc/[pid]/status`. On Windows, can be set to the base name of `GetProcessImageFileNameW`. + column: String + examples: + - otelcol + type: string + process.executable.path: + brief: | + The full path to the process executable. On Linux based systems, can be set to the target of `proc/[pid]/exe`. On Windows, can be set to the result of `GetProcessImageFileNameW`. + column: String + examples: + - /usr/bin/cmd/otelcol + type: string + process.owner: + brief: | + The username of the user that owns the process. + column: String + examples: + - root + type: string + process.parent_pid: + brief: | + Parent Process identifier (PID). + column: Int64 + examples: + - 111 + type: int + process.pid: + brief: | + Process identifier (PID). + column: Int64 + examples: + - 1234 + type: int + process.runtime.description: + brief: | + An additional description about the runtime of the process, for example a specific vendor customization of the runtime environment. + column: String + examples: + - Eclipse OpenJ9 Eclipse OpenJ9 VM openj9-0.21.0 + type: string + process.runtime.name: + brief: | + The name of the runtime of this process. For compiled native binaries, this SHOULD be the name of the compiler. + column: String + examples: + - OpenJDK Runtime Environment + type: string + process.runtime.version: + brief: | + The version of the runtime of this process, as returned by the runtime without modification. + column: String + examples: + - 14.0.2 + type: string + rpc.connect_rpc.error_code: + brief: The [error codes](https://connect.build/docs/protocol/#error-codes) of + the Connect request. Error codes are always string values. + column: Enum8(0 = 'cancelled', 1 = 'unknown', 2 = 'invalid_argument', 3 = 'deadline_exceeded', + 4 = 'not_found', 5 = 'already_exists', 6 = 'permission_denied', 7 = 'resource_exhausted', + 8 = 'failed_precondition', 9 = 'aborted', 10 = 'out_of_range', 11 = 'unimplemented', + 12 = 'internal', 13 = 'unavailable', 14 = 'data_loss', 15 = 'unauthenticated') + enum: + - cancelled + - unknown + - invalid_argument + - deadline_exceeded + - not_found + - already_exists + - permission_denied + - resource_exhausted + - failed_precondition + - aborted + - out_of_range + - unimplemented + - internal + - unavailable + - data_loss + - unauthenticated + type: string + rpc.connect_rpc.request.metadata: + brief: | + Connect request metadata, `` being the normalized Connect Metadata key (lowercase), the value being the metadata values. + column: Array(String) + examples: + - rpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"] + type: template[string[]] + rpc.connect_rpc.response.metadata: + brief: | + Connect response metadata, `` being the normalized Connect Metadata key (lowercase), the value being the metadata values. + column: Array(String) + examples: + - rpc.response.metadata.my-custom-metadata-attribute=["attribute_value"] + type: template[string[]] + rpc.grpc.request.metadata: + brief: | + gRPC request metadata, `` being the normalized gRPC Metadata key (lowercase), the value being the metadata values. + column: Array(String) + examples: + - rpc.grpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"] + type: template[string[]] + rpc.grpc.response.metadata: + brief: | + gRPC response metadata, `` being the normalized gRPC Metadata key (lowercase), the value being the metadata values. + column: Array(String) + examples: + - rpc.grpc.response.metadata.my-custom-metadata-attribute=["attribute_value"] + type: template[string[]] + rpc.grpc.status_code: + brief: The [numeric status code](https://github.com/grpc/grpc/blob/v1.33.2/doc/statuscodes.md) + of the gRPC request. + column: UInt8 + enum: + - 0 + - 1 + - 2 + - 3 + - 4 + - 5 + - 6 + - 7 + - 8 + - 9 + - 10 + - 11 + - 12 + - 13 + - 14 + - 15 + - 16 + type: int + rpc.jsonrpc.error_code: + brief: '`error.code` property of response if it is an error response.' + column: Int64 + examples: + - -32700 + - 100 + type: int + rpc.jsonrpc.error_message: + brief: '`error.message` property of response if it is an error response.' + column: String + examples: + - Parse error + - User already exists + type: string + rpc.jsonrpc.request_id: + brief: | + `id` property of request or response. Since protocol allows id to be int, string, `null` or missing (for notifications), value is expected to be cast to string for simplicity. Use empty string in case of `null` value. Omit entirely if this is a notification. + column: String + examples: + - "10" + - request-7 + - "" + type: string + rpc.jsonrpc.version: + brief: Protocol version as in `jsonrpc` property of request/response. Since JSON-RPC + 1.0 doesn't specify this, the value can be omitted. + column: String + examples: + - "2.0" + - "1.0" + type: string + rpc.method: + brief: The name of the (logical) method being called, must be equal to the $method + part in the span name. + column: String + examples: + - exampleMethod + type: string + rpc.service: + brief: The full (logical) name of the service being called, including its package + name, if applicable. + column: String + examples: + - myservice.EchoService + type: string + rpc.system: + brief: A string identifying the remoting system. See below for a list of well-known + identifiers. + column: String + type: string + server.address: + brief: Server domain name if available without reverse DNS lookup; otherwise, + IP address or Unix domain socket name. + column: String + examples: + - example.com + - 10.1.2.80 + - /tmp/my.sock + type: string + server.port: + brief: Server port number. + column: UInt16 + examples: + - 80 + - 8080 + - 443 + type: int + service.instance.id: + brief: | + The string ID of the service instance. + column: String + examples: + - my-k8s-pod-deployment-1 + - 627cc493-f310-47de-96bd-71410b7dec09 + type: string + service.name: + brief: | + Logical name of the service. + column: String + examples: + - shoppingcart + type: string + service.namespace: + brief: | + A namespace for `service.name`. + column: String + examples: + - Shop + type: string + service.version: + brief: | + The version string of the service API or implementation. The format is not defined by these conventions. + column: String + examples: + - 2.0.0 + - a01dbef8a + type: string + session.id: + brief: A unique id to identify a session. + column: String + examples: + - 00112233-4455-6677-8899-aabbccddeeff + type: string + session.previous_id: + brief: The previous `session.id` for this user, when known. + column: String + examples: + - 00112233-4455-6677-8899-aabbccddeeff + type: string + source.address: + brief: Source address - domain name if available without reverse DNS lookup; otherwise, + IP address or Unix domain socket name. + column: String + examples: + - source.example.com + - 10.1.2.80 + - /tmp/my.sock + type: string + source.port: + brief: Source port number + column: UInt16 + examples: + - 3389 + - 2888 + type: int + state: + brief: The state of a connection in the pool + column: Enum8(0 = 'idle', 1 = 'used') + enum: + - idle + - used + type: string + system.cpu.logical_number: + brief: The logical CPU number [0..n-1] + column: Int64 + examples: + - 1 + type: int + system.cpu.state: + brief: The state of the CPU + column: String + examples: + - idle + - interrupt + type: string + system.device: + brief: The device identifier + column: String + examples: + - (identifier) + type: string + system.filesystem.mode: + brief: The filesystem mode + column: String + examples: + - rw, ro + type: string + system.filesystem.mountpoint: + brief: The filesystem mount path + column: String + examples: + - /mnt/data + type: string + system.filesystem.state: + brief: The filesystem state + column: Enum8(0 = 'used', 1 = 'free', 2 = 'reserved') + enum: + - used + - free + - reserved + type: string + system.filesystem.type: + brief: The filesystem type + column: String + examples: + - ext4 + type: string + system.memory.state: + brief: The memory state + column: String + examples: + - free + - cached + type: string + system.network.state: + brief: A stateless protocol MUST NOT set this attribute + column: Enum8(0 = 'close', 1 = 'close_wait', 2 = 'closing', 3 = 'delete', 4 = + 'established', 5 = 'fin_wait_1', 6 = 'fin_wait_2', 7 = 'last_ack', 8 = 'listen', + 9 = 'syn_recv', 10 = 'syn_sent', 11 = 'time_wait') + enum: + - close + - close_wait + - closing + - delete + - established + - fin_wait_1 + - fin_wait_2 + - last_ack + - listen + - syn_recv + - syn_sent + - time_wait + type: string + system.paging.direction: + brief: The paging access direction + column: Enum8(0 = 'in', 1 = 'out') + enum: + - in + - out + type: string + system.paging.state: + brief: The memory paging state + column: Enum8(0 = 'used', 1 = 'free') + enum: + - used + - free + type: string + system.paging.type: + brief: The memory paging type + column: Enum8(0 = 'major', 1 = 'minor') + enum: + - major + - minor + type: string + system.processes.status: + brief: | + The process state, e.g., [Linux Process State Codes](https://man7.org/linux/man-pages/man1/ps.1.html#PROCESS_STATE_CODES) + column: String + examples: + - running + type: string + telemetry.distro.name: + brief: | + The name of the auto instrumentation agent or distribution, if used. + column: String + examples: + - parts-unlimited-java + type: string + telemetry.distro.version: + brief: | + The version string of the auto instrumentation agent or distribution, if used. + column: String + examples: + - 1.2.3 + type: string + telemetry.sdk.language: + brief: | + The language of the telemetry SDK. + column: String + type: string + telemetry.sdk.name: + brief: | + The name of the telemetry SDK as defined above. + column: String + examples: + - opentelemetry + type: string + telemetry.sdk.version: + brief: | + The version string of the telemetry SDK. + column: String + examples: + - 1.2.3 + type: string + thread.id: + brief: | + Current "managed" thread ID (as opposed to OS thread ID). + column: Int64 + examples: + - 42 + type: int + thread.name: + brief: | + Current thread name. + column: String + examples: + - main + type: string + tls.cipher: + brief: | + String indicating the [cipher](https://datatracker.ietf.org/doc/html/rfc5246#appendix-A.5) used during the current connection. + column: String + examples: + - TLS_RSA_WITH_3DES_EDE_CBC_SHA + - TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256 + type: string + tls.client.certificate: + brief: | + PEM-encoded stand-alone certificate offered by the client. This is usually mutually-exclusive of `client.certificate_chain` since this value also exists in that list. + column: String + examples: + - MII... + type: string + tls.client.certificate_chain: + brief: | + Array of PEM-encoded certificates that make up the certificate chain offered by the client. This is usually mutually-exclusive of `client.certificate` since that value should be the first certificate in the chain. + column: Array(String) + examples: + - MII... + - MI... + type: string[] + tls.client.hash.md5: + brief: | + Certificate fingerprint using the MD5 digest of DER-encoded version of certificate offered by the client. For consistency with other hash values, this value should be formatted as an uppercase hash. + column: String + examples: + - 0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC + type: string + tls.client.hash.sha1: + brief: | + Certificate fingerprint using the SHA1 digest of DER-encoded version of certificate offered by the client. For consistency with other hash values, this value should be formatted as an uppercase hash. + column: String + examples: + - 9E393D93138888D288266C2D915214D1D1CCEB2A + type: string + tls.client.hash.sha256: + brief: | + Certificate fingerprint using the SHA256 digest of DER-encoded version of certificate offered by the client. For consistency with other hash values, this value should be formatted as an uppercase hash. + column: String + examples: + - 0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0 + type: string + tls.client.issuer: + brief: Distinguished name of [subject](https://datatracker.ietf.org/doc/html/rfc5280#section-4.1.2.6) + of the issuer of the x.509 certificate presented by the client. + column: String + examples: + - CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com + type: string + tls.client.ja3: + brief: A hash that identifies clients based on how they perform an SSL/TLS handshake. + column: String + examples: + - d4e5b18d6b55c71272893221c96ba240 + type: string + tls.client.not_after: + brief: Date/Time indicating when client certificate is no longer considered valid. + column: String + examples: + - "2021-01-01T00:00:00.000Z" + type: string + tls.client.not_before: + brief: Date/Time indicating when client certificate is first considered valid. + column: String + examples: + - "1970-01-01T00:00:00.000Z" + type: string + tls.client.server_name: + brief: Also called an SNI, this tells the server which hostname to which the client + is attempting to connect to. + column: String + examples: + - opentelemetry.io + type: string + tls.client.subject: + brief: Distinguished name of subject of the x.509 certificate presented by the + client. + column: String + examples: + - CN=myclient, OU=Documentation Team, DC=example, DC=com + type: string + tls.client.supported_ciphers: + brief: Array of ciphers offered by the client during the client hello. + column: Array(String) + examples: + - '"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", + "..."' + type: string[] + tls.curve: + brief: String indicating the curve used for the given cipher, when applicable + column: String + examples: + - secp256r1 + type: string + tls.established: + brief: Boolean flag indicating if the TLS negotiation was successful and transitioned + to an encrypted tunnel. + column: Bool + examples: + - true + type: boolean + tls.next_protocol: + brief: | + String indicating the protocol being tunneled. Per the values in the [IANA registry](https://www.iana.org/assignments/tls-extensiontype-values/tls-extensiontype-values.xhtml#alpn-protocol-ids), this string should be lower case. + column: String + examples: + - http/1.1 + type: string + tls.protocol.name: + brief: | + Normalized lowercase protocol name parsed from original string of the negotiated [SSL/TLS protocol version](https://www.openssl.org/docs/man1.1.1/man3/SSL_get_version.html#RETURN-VALUES) + column: String + type: string + tls.protocol.version: + brief: | + Numeric part of the version parsed from the original string of the negotiated [SSL/TLS protocol version](https://www.openssl.org/docs/man1.1.1/man3/SSL_get_version.html#RETURN-VALUES) + column: String + examples: + - "1.2" + - "3" + type: string + tls.resumed: + brief: Boolean flag indicating if this TLS connection was resumed from an existing + TLS negotiation. + column: Bool + examples: + - true + type: boolean + tls.server.certificate: + brief: | + PEM-encoded stand-alone certificate offered by the server. This is usually mutually-exclusive of `server.certificate_chain` since this value also exists in that list. + column: String + examples: + - MII... + type: string + tls.server.certificate_chain: + brief: | + Array of PEM-encoded certificates that make up the certificate chain offered by the server. This is usually mutually-exclusive of `server.certificate` since that value should be the first certificate in the chain. + column: Array(String) + examples: + - MII... + - MI... + type: string[] + tls.server.hash.md5: + brief: | + Certificate fingerprint using the MD5 digest of DER-encoded version of certificate offered by the server. For consistency with other hash values, this value should be formatted as an uppercase hash. + column: String + examples: + - 0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC + type: string + tls.server.hash.sha1: + brief: | + Certificate fingerprint using the SHA1 digest of DER-encoded version of certificate offered by the server. For consistency with other hash values, this value should be formatted as an uppercase hash. + column: String + examples: + - 9E393D93138888D288266C2D915214D1D1CCEB2A + type: string + tls.server.hash.sha256: + brief: | + Certificate fingerprint using the SHA256 digest of DER-encoded version of certificate offered by the server. For consistency with other hash values, this value should be formatted as an uppercase hash. + column: String + examples: + - 0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0 + type: string + tls.server.issuer: + brief: Distinguished name of [subject](https://datatracker.ietf.org/doc/html/rfc5280#section-4.1.2.6) + of the issuer of the x.509 certificate presented by the client. + column: String + examples: + - CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com + type: string + tls.server.ja3s: + brief: A hash that identifies servers based on how they perform an SSL/TLS handshake. + column: String + examples: + - d4e5b18d6b55c71272893221c96ba240 + type: string + tls.server.not_after: + brief: Date/Time indicating when server certificate is no longer considered valid. + column: String + examples: + - "2021-01-01T00:00:00.000Z" + type: string + tls.server.not_before: + brief: Date/Time indicating when server certificate is first considered valid. + column: String + examples: + - "1970-01-01T00:00:00.000Z" + type: string + tls.server.subject: + brief: Distinguished name of subject of the x.509 certificate presented by the + server. + column: String + examples: + - CN=myserver, OU=Documentation Team, DC=example, DC=com + type: string + url.fragment: + brief: The [URI fragment](https://www.rfc-editor.org/rfc/rfc3986#section-3.5) + component + column: String + examples: + - SemConv + type: string + url.full: + brief: Absolute URL describing a network resource according to [RFC3986](https://www.rfc-editor.org/rfc/rfc3986) + column: String + examples: + - https://www.foo.bar/search?q=OpenTelemetry#SemConv + - //localhost + type: string + url.path: + brief: The [URI path](https://www.rfc-editor.org/rfc/rfc3986#section-3.3) component + column: String + examples: + - /search + type: string + url.query: + brief: The [URI query](https://www.rfc-editor.org/rfc/rfc3986#section-3.4) component + column: String + examples: + - q=OpenTelemetry + type: string + url.scheme: + brief: The [URI scheme](https://www.rfc-editor.org/rfc/rfc3986#section-3.1) component + identifying the used protocol. + column: String + examples: + - https + - ftp + - telnet + type: string + user_agent.original: + brief: | + Value of the [HTTP User-Agent](https://www.rfc-editor.org/rfc/rfc9110.html#field.user-agent) header sent by the client. + column: String + examples: + - CERN-LineMode/2.15 libwww/2.17b3 + - Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 + (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1 + type: string + webengine.description: + brief: | + Additional description of the web engine (e.g. detailed version and edition information). + column: String + examples: + - WildFly Full 21.0.0.Final (WildFly Core 13.0.1.Final) - 2.2.2.Final + type: string + webengine.name: + brief: | + The name of the web engine. + column: String + examples: + - WildFly + type: string + webengine.version: + brief: | + The version of the web engine. + column: String + examples: + - 21.0.0 + type: string +statistics: + enum: 23 + total: 716 + unknown: 0 diff --git a/internal/otelschema/_testdata/model/README.md b/internal/otelschema/_testdata/model/README.md deleted file mode 100644 index 2fc6f39b..00000000 --- a/internal/otelschema/_testdata/model/README.md +++ /dev/null @@ -1,41 +0,0 @@ -# YAML Model for Semantic Conventions - -The YAML descriptions of semantic convention contained in this directory are intended to -be used by the various OpenTelemetry language implementations to aid in automatic -generation of semantics-related code. - -⚠ If you want to read the semantic conventions and not edit them, please see -the generated markdown output in the [docs](../docs/README.md) folder. - -## Writing semantic conventions - -Semantic conventions for the spec MUST adhere to the -[attribute naming](../docs/general/attribute-naming.md), -[attribute requirement level](../docs/general/attribute-requirement-level.md), -and [metric requirement level](https://github.com/open-telemetry/opentelemetry-specification/tree/v1.26.0/specification/metrics/metric-requirement-level.md) conventions. - -Refer to the [syntax](https://github.com/open-telemetry/build-tools/tree/v0.23.0/semantic-conventions/syntax.md) -for how to write the YAML files for semantic conventions and what the YAML properties mean. - -A schema file for VS code is configured in the `/.vscode/settings.json` of this -repository, enabling auto-completion and additional checks. Refer to -[the generator README](https://github.com/open-telemetry/build-tools/tree/v0.23.0/semantic-conventions/README.md) for what extension you need. - -## Generating markdown - -These YAML files are used by the make target `table-generation` to generate consistently -formatted Markdown tables for all semantic conventions in the specification. Run it from the root of this repository using the command - -``` -make table-generation -``` - -For more information, see the [semantic convention generator](https://github.com/open-telemetry/build-tools/tree/v0.23.0/semantic-conventions) -in the OpenTelemetry build tools repository. -Using this build tool, it is also possible to generate code for use in OpenTelemetry -language projects. - -See also: - -* [Markdown Tables](https://github.com/open-telemetry/build-tools/tree/main/semantic-conventions#markdown-tables) -* [Code Generator](https://github.com/open-telemetry/build-tools/tree/main/semantic-conventions#code-generator) diff --git a/internal/otelschema/_testdata/model/exception.yaml b/internal/otelschema/_testdata/model/exception.yaml deleted file mode 100644 index 9f47fb67..00000000 --- a/internal/otelschema/_testdata/model/exception.yaml +++ /dev/null @@ -1,33 +0,0 @@ -groups: - - id: exception - type: span - prefix: exception - brief: > - This document defines the shared attributes used to - report a single exception associated with a span or log. - attributes: - - id: type - type: string - brief: > - The type of the exception (its fully-qualified class name, if applicable). - The dynamic type of the exception should be preferred over the static type - in languages that support it. - examples: ["java.net.ConnectException", "OSError"] - - id: message - type: string - brief: The exception message. - examples: ["Division by zero", "Can't convert 'int' object to str implicitly"] - - id: stacktrace - type: string - brief: > - A stacktrace as a string in the natural representation for the language runtime. - The representation is to be determined and documented by each language SIG. - examples: 'Exception in thread "main" java.lang.RuntimeException: Test exception\n - at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n - at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n - at com.example.GenerateTrace.main(GenerateTrace.java:5)' - - constraints: - - any_of: - - "exception.type" - - "exception.message" diff --git a/internal/otelschema/_testdata/model/faas-common.yaml b/internal/otelschema/_testdata/model/faas-common.yaml deleted file mode 100644 index 7f903bb5..00000000 --- a/internal/otelschema/_testdata/model/faas-common.yaml +++ /dev/null @@ -1,77 +0,0 @@ -groups: - - id: attributes.faas.common - type: attribute_group - brief: "Describes FaaS attributes." - prefix: faas - attributes: - - id: trigger - brief: 'Type of the trigger which caused this function invocation.' - type: - allow_custom_values: false - members: - - id: datasource - value: 'datasource' - brief: 'A response to some data source operation such as a database or filesystem read/write' - - id: http - value: 'http' - brief: 'To provide an answer to an inbound HTTP request' - - id: pubsub - value: 'pubsub' - brief: 'A function is set to be executed when messages are sent to a messaging system' - - id: timer - value: 'timer' - brief: 'A function is scheduled to be executed regularly' - - id: other - value: 'other' - brief: 'If none of the others apply' - - id: invoked_name - type: string - requirement_level: required - brief: > - The name of the invoked function. - note: > - SHOULD be equal to the `faas.name` resource attribute of the - invoked function. - examples: 'my-function' - - id: invoked_provider - type: - allow_custom_values: true - members: - - id: 'alibaba_cloud' - value: 'alibaba_cloud' - brief: 'Alibaba Cloud' - - id: 'aws' - value: 'aws' - brief: 'Amazon Web Services' - - id: 'azure' - value: 'azure' - brief: 'Microsoft Azure' - - id: 'gcp' - value: 'gcp' - brief: 'Google Cloud Platform' - - id: 'tencent_cloud' - value: 'tencent_cloud' - brief: 'Tencent Cloud' - requirement_level: required - brief: > - The cloud provider of the invoked function. - note: > - SHOULD be equal to the `cloud.provider` resource attribute of the - invoked function. - - id: invoked_region - type: string - requirement_level: - conditionally_required: > - For some cloud providers, like AWS or GCP, the region in which a - function is hosted is essential to uniquely identify the function - and also part of its endpoint. Since it's part of the endpoint - being called, the region is always known to clients. In these cases, - `faas.invoked_region` MUST be set accordingly. If the region is - unknown to the client or not required for identifying the invoked - function, setting `faas.invoked_region` is optional. - brief: > - The cloud region of the invoked function. - note: > - SHOULD be equal to the `cloud.region` resource attribute of the - invoked function. - examples: 'eu-central-1' diff --git a/internal/otelschema/_testdata/model/general.yaml b/internal/otelschema/_testdata/model/general.yaml deleted file mode 100644 index e3bad3d4..00000000 --- a/internal/otelschema/_testdata/model/general.yaml +++ /dev/null @@ -1,85 +0,0 @@ -groups: - - id: general.client - type: attribute_group - brief: > - General client attributes. - attributes: - - ref: client.address - - ref: client.port - - id: general.server - type: attribute_group - brief: > - General server attributes. - attributes: - - ref: server.address - - ref: server.port - - id: general.source - type: attribute_group - brief: > - General source attributes. - attributes: - - ref: source.address - - ref: source.port - - id: general.destination - type: attribute_group - brief: > - General destination attributes. - attributes: - - ref: destination.address - - ref: destination.port - - id: peer - prefix: peer - type: span - brief: "Operations that access some remote service." - attributes: - - id: service - type: string - brief: > - The [`service.name`](/docs/resource/README.md#service) - of the remote service. SHOULD be equal to the actual `service.name` - resource attribute of the remote service if any. - examples: "AuthTokenCache" - - id: identity - prefix: enduser - type: span - brief: > - These attributes may be used for any operation with an authenticated and/or authorized enduser. - attributes: - - id: id - type: string - brief: > - Username or client_id extracted from the access token or - [Authorization](https://tools.ietf.org/html/rfc7235#section-4.2) - header in the inbound request from outside the system. - examples: 'username' - - id: role - type: string - brief: 'Actual/assumed role the client is making the request under extracted from token or application security context.' - examples: 'admin' - - id: scope - type: string - brief: > - Scopes or granted authorities the client currently possesses extracted from token - or application security context. The value would come from the scope associated - with an [OAuth 2.0 Access Token](https://tools.ietf.org/html/rfc6749#section-3.3) - or an attribute value in a [SAML 2.0 Assertion](http://docs.oasis-open.org/security/saml/Post2.0/sstc-saml-tech-overview-2.0.html). - examples: 'read:message, write:files' - - id: thread - type: span - brief: > - These attributes may be used for any operation to store information about a thread that started a span. - attributes: - - ref: thread.id - - ref: thread.name - - id: code - type: span - brief: > - These attributes allow to report this unit of code and therefore to provide more context about the span. - attributes: - - ref: code.function - - ref: code.namespace - - ref: code.filepath - - ref: code.lineno - - ref: code.column - - ref: code.stacktrace - requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/http-common.yaml b/internal/otelschema/_testdata/model/http-common.yaml deleted file mode 100644 index 2187943d..00000000 --- a/internal/otelschema/_testdata/model/http-common.yaml +++ /dev/null @@ -1,87 +0,0 @@ -groups: - - id: attributes.http.common - type: attribute_group - brief: "Describes HTTP attributes." - attributes: - - ref: http.request.method - requirement_level: required - - ref: http.response.status_code - requirement_level: - conditionally_required: If and only if one was received/sent. - - ref: error.type - requirement_level: - conditionally_required: If request has ended with an error. - examples: ['timeout', 'java.net.UnknownHostException', 'server_certificate_invalid', '500'] - note: | - If the request fails with an error before response status code was sent or received, - `error.type` SHOULD be set to exception type (its fully-qualified class name, if applicable) - or a component-specific low cardinality error identifier. - - If response status code was sent or received and status indicates an error according to [HTTP span status definition](/docs/http/http-spans.md), - `error.type` SHOULD be set to the status code number (represented as a string), an exception type (if thrown) or a component-specific error identifier. - - The `error.type` value SHOULD be predictable and SHOULD have low cardinality. - Instrumentations SHOULD document the list of errors they report. - - The cardinality of `error.type` within one instrumentation library SHOULD be low, but - telemetry consumers that aggregate data from multiple instrumentation libraries and applications - should be prepared for `error.type` to have high cardinality at query time, when no - additional filters are applied. - - If the request has completed successfully, instrumentations SHOULD NOT set `error.type`. - - ref: network.protocol.name - examples: ['http', 'spdy'] - requirement_level: - conditionally_required: If not `http` and `network.protocol.version` is set. - - ref: network.protocol.version - examples: ['1.0', '1.1', '2', '3'] - - - id: attributes.http.client - type: attribute_group - brief: 'HTTP Client attributes' - extends: attributes.http.common - attributes: - - ref: server.address - requirement_level: required - brief: > - Host identifier of the ["URI origin"](https://www.rfc-editor.org/rfc/rfc9110.html#name-uri-origin) HTTP request is sent to. - note: > - If an HTTP client request is explicitly made to an IP address, e.g. `http://x.x.x.x:8080`, then - `server.address` SHOULD be the IP address `x.x.x.x`. A DNS lookup SHOULD NOT be used. - - ref: server.port - requirement_level: required - brief: > - Port identifier of the ["URI origin"](https://www.rfc-editor.org/rfc/rfc9110.html#name-uri-origin) HTTP request is sent to. - - ref: url.scheme - requirement_level: opt_in - examples: ["http", "https"] - - - id: attributes.http.server - type: attribute_group - brief: 'HTTP Server attributes' - extends: attributes.http.common - attributes: - - ref: http.route - requirement_level: - conditionally_required: If and only if it's available - - ref: server.address - brief: > - Name of the local HTTP server that received the request. - note: > - See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). - - ref: server.port - brief: > - Port of the local HTTP server that received the request. - note: > - See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). - requirement_level: - conditionally_required: If `server.address` is set. - - ref: url.scheme - requirement_level: required - examples: ["http", "https"] - note: > - The scheme of the original client request, if known - (e.g. from [Forwarded#proto](https://developer.mozilla.org/docs/Web/HTTP/Headers/Forwarded#proto), - [X-Forwarded-Proto](https://developer.mozilla.org/docs/Web/HTTP/Headers/X-Forwarded-Proto), - or a similar header). - Otherwise, the scheme of the immediate peer request. diff --git a/internal/otelschema/_testdata/model/logs/events.yaml b/internal/otelschema/_testdata/model/logs/events.yaml deleted file mode 100644 index 3942d432..00000000 --- a/internal/otelschema/_testdata/model/logs/events.yaml +++ /dev/null @@ -1,18 +0,0 @@ -groups: - - id: event - type: attribute_group - prefix: event - brief: > - This document defines attributes for Events represented using Log Records. - attributes: - - id: name - type: string - requirement_level: required - brief: > - Identifies the class / type of event. - note: > - Event names are subject to the same rules as [attribute names](https://github.com/open-telemetry/opentelemetry-specification/tree/v1.26.0/specification/common/attribute-naming.md). - Notably, event names are namespaced to avoid collisions and provide a clean - separation of semantics for events in separate domains like browser, mobile, and - kubernetes. - examples: ['browser.mouse.click', 'device.app.lifecycle'] diff --git a/internal/otelschema/_testdata/model/logs/general.yaml b/internal/otelschema/_testdata/model/logs/general.yaml deleted file mode 100644 index 48da970c..00000000 --- a/internal/otelschema/_testdata/model/logs/general.yaml +++ /dev/null @@ -1,19 +0,0 @@ -groups: - - id: log.record - prefix: log.record - type: attribute_group - brief: > - The attributes described in this section are rather generic. They may be used in any Log Record they apply to. - attributes: - - id: uid - type: string - requirement_level: opt_in - brief: > - A unique identifier for the Log Record. - note: > - If an id is provided, other log records with the same id will be considered duplicates and can be removed safely. - This means, that two distinguishable log records MUST have different values. - - The id MAY be an [Universally Unique Lexicographically Sortable Identifier (ULID)](https://github.com/ulid/spec), - but other identifiers (e.g. UUID) may be used as needed. - examples: ["01ARZ3NDEKTSV4RRFFQ69G5FAV"] diff --git a/internal/otelschema/_testdata/model/logs/log-exception.yaml b/internal/otelschema/_testdata/model/logs/log-exception.yaml deleted file mode 100644 index d0b25155..00000000 --- a/internal/otelschema/_testdata/model/logs/log-exception.yaml +++ /dev/null @@ -1,16 +0,0 @@ -groups: - - id: log-exception - type: attribute_group - prefix: exception - brief: > - This document defines attributes for exceptions represented using Log - Records. - attributes: - - ref: exception.type - - ref: exception.message - - ref: exception.stacktrace - - constraints: - - any_of: - - "exception.type" - - "exception.message" diff --git a/internal/otelschema/_testdata/model/logs/log-feature_flag.yaml b/internal/otelschema/_testdata/model/logs/log-feature_flag.yaml deleted file mode 100644 index 6d902423..00000000 --- a/internal/otelschema/_testdata/model/logs/log-feature_flag.yaml +++ /dev/null @@ -1,11 +0,0 @@ -groups: - - id: log-feature_flag - type: event - prefix: feature_flag - brief: > - This document defines attributes for feature flag evaluations - represented using Log Records. - attributes: - - ref: feature_flag.key - - ref: feature_flag.provider_name - - ref: feature_flag.variant diff --git a/internal/otelschema/_testdata/model/logs/media.yaml b/internal/otelschema/_testdata/model/logs/media.yaml deleted file mode 100644 index b8d1c299..00000000 --- a/internal/otelschema/_testdata/model/logs/media.yaml +++ /dev/null @@ -1,49 +0,0 @@ -groups: - - id: attributes.log - prefix: log - type: attribute_group - brief: "Describes Log attributes" - attributes: - - id: iostream - requirement_level: opt_in - brief: > - The stream associated with the log. See below for a list of well-known values. - type: - allow_custom_values: false - members: - - id: stdout - value: 'stdout' - brief: 'Logs from stdout stream' - - id: stderr - value: 'stderr' - brief: 'Events from stderr stream' - - id: attributes.log.file - prefix: log.file - type: attribute_group - brief: > - A file to which log was emitted. - attributes: - - id: name - type: string - requirement_level: recommended - brief: > - The basename of the file. - examples: ["audit.log"] - - id: path - type: string - requirement_level: opt_in - brief: > - The full path to the file. - examples: [ "/var/log/mysql/audit.log" ] - - id: name_resolved - type: string - requirement_level: opt_in - brief: > - The basename of the file, with symlinks resolved. - examples: [ "uuid.log" ] - - id: path_resolved - type: string - requirement_level: opt_in - brief: > - The full path to the file, with symlinks resolved. - examples: [ "/var/lib/docker/uuid.log" ] diff --git a/internal/otelschema/_testdata/model/logs/mobile-events.yaml b/internal/otelschema/_testdata/model/logs/mobile-events.yaml deleted file mode 100644 index d6f49e70..00000000 --- a/internal/otelschema/_testdata/model/logs/mobile-events.yaml +++ /dev/null @@ -1,72 +0,0 @@ -groups: - - id: ios.lifecycle.events - type: event - prefix: ios - name: device.app.lifecycle - brief: > - This event represents an occurrence of a lifecycle transition on the iOS platform. - attributes: - - id: state - requirement_level: "required" - note: > - The iOS lifecycle states are defined in the [UIApplicationDelegate documentation](https://developer.apple.com/documentation/uikit/uiapplicationdelegate#1656902), - and from which the `OS terminology` column values are derived. - brief: > - This attribute represents the state the application has transitioned into at the occurrence of the event. - type: - allow_custom_values: false - members: - - id: active - value: 'active' - brief: > - The app has become `active`. Associated with UIKit notification `applicationDidBecomeActive`. - - id: inactive - value: 'inactive' - brief: > - The app is now `inactive`. Associated with UIKit notification `applicationWillResignActive`. - - id: background - value: 'background' - brief: > - The app is now in the background. - This value is associated with UIKit notification `applicationDidEnterBackground`. - - id: foreground - value: 'foreground' - brief: > - The app is now in the foreground. - This value is associated with UIKit notification `applicationWillEnterForeground`. - - id: terminate - value: 'terminate' - brief: > - The app is about to terminate. Associated with UIKit notification `applicationWillTerminate`. - - id: android.lifecycle.events - type: event - prefix: android - name: device.app.lifecycle - brief: > - This event represents an occurrence of a lifecycle transition on the Android platform. - attributes: - - id: state - requirement_level: required - brief: > - This attribute represents the state the application has transitioned into at the occurrence of the event. - note: > - The Android lifecycle states are defined in [Activity lifecycle callbacks](https://developer.android.com/guide/components/activities/activity-lifecycle#lc), - and from which the `OS identifiers` are derived. - type: - allow_custom_values: false - members: - - id: created - value: 'created' - brief: > - Any time before Activity.onResume() or, if the app has no Activity, Context.startService() - has been called in the app for the first time. - - id: background - value: 'background' - brief: > - Any time after Activity.onPause() or, if the app has no Activity, - Context.stopService() has been called when the app was in the foreground state. - - id: foreground - value: 'foreground' - brief: > - Any time after Activity.onResume() or, if the app has no Activity, - Context.startService() has been called when the app was in either the created or background states. diff --git a/internal/otelschema/_testdata/model/metrics/database-metrics.yaml b/internal/otelschema/_testdata/model/metrics/database-metrics.yaml deleted file mode 100644 index 3df6d86a..00000000 --- a/internal/otelschema/_testdata/model/metrics/database-metrics.yaml +++ /dev/null @@ -1,107 +0,0 @@ -groups: - - id: attributes.db - type: attribute_group - brief: Describes Database attributes - attributes: - - id: state - type: - allow_custom_values: false - members: - - id: idle - value: 'idle' - - id: used - value: 'used' - requirement_level: required - brief: "The state of a connection in the pool" - examples: ["idle"] - - id: pool.name - type: string - requirement_level: required - brief: > - The name of the connection pool; unique within the instrumented application. - In case the connection pool implementation doesn't provide a name, - then the [db.connection_string](/docs/database/database-spans.md#connection-level-attributes) - should be used - examples: ["myDataSource"] - - - id: metric.db.client.connections.usage - type: metric - metric_name: db.client.connections.usage - brief: "The number of connections that are currently in state described by the `state` attribute" - instrument: updowncounter - unit: "{connection}" - attributes: - - ref: state - - ref: pool.name - - - id: metric.db.client.connections.idle.max - type: metric - metric_name: db.client.connections.idle.max - brief: "The maximum number of idle open connections allowed" - instrument: updowncounter - unit: "{connection}" - attributes: - - ref: pool.name - - - id: metric.db.client.connections.idle.min - type: metric - metric_name: db.client.connections.idle.min - brief: "The minimum number of idle open connections allowed" - instrument: updowncounter - unit: "{connection}" - attributes: - - ref: pool.name - - - id: metric.db.client.connections.max - type: metric - metric_name: db.client.connections.max - brief: "The maximum number of open connections allowed" - instrument: updowncounter - unit: "{connection}" - attributes: - - ref: pool.name - - - id: metric.db.client.connections.pending_requests - type: metric - metric_name: db.client.connections.pending_requests - brief: "The number of pending requests for an open connection, cumulative for the entire pool" - instrument: updowncounter - unit: "{request}" - attributes: - - ref: pool.name - - - id: metric.db.client.connections.timeouts - type: metric - metric_name: db.client.connections.timeouts - brief: "The number of connection timeouts that have occurred trying to obtain a connection from the pool" - instrument: counter - unit: "{timeout}" - attributes: - - ref: pool.name - - - id: metric.db.client.connections.create_time - type: metric - metric_name: db.client.connections.create_time - brief: "The time it took to create a new connection" - instrument: histogram - unit: "ms" - attributes: - - ref: pool.name - - - id: metric.db.client.connections.wait_time - type: metric - metric_name: db.client.connections.wait_time - brief: "The time it took to obtain an open connection from the pool" - instrument: histogram - unit: "ms" - attributes: - - ref: pool.name - - - id: metric.db.client.connections.use_time - type: metric - metric_name: db.client.connections.use_time - brief: "The time between borrowing a connection and returning it to the pool" - instrument: histogram - unit: "ms" - attributes: - - ref: pool.name diff --git a/internal/otelschema/_testdata/model/metrics/faas-metrics.yaml b/internal/otelschema/_testdata/model/metrics/faas-metrics.yaml deleted file mode 100644 index d68119f3..00000000 --- a/internal/otelschema/_testdata/model/metrics/faas-metrics.yaml +++ /dev/null @@ -1,81 +0,0 @@ -groups: - - id: metric.faas.invoke_duration - type: metric - metric_name: faas.invoke_duration - brief: "Measures the duration of the function's logic execution" - instrument: histogram - unit: "s" - attributes: - - ref: faas.trigger - - - id: metric.faas.init_duration - type: metric - metric_name: faas.init_duration - brief: "Measures the duration of the function's initialization, such as a cold start" - instrument: histogram - unit: "s" - attributes: - - ref: faas.trigger - - - id: metric.faas.coldstarts - type: metric - metric_name: faas.coldstarts - brief: "Number of invocation cold starts" - instrument: counter - unit: "{coldstart}" - attributes: - - ref: faas.trigger - - - id: metric.faas.errors - type: metric - metric_name: faas.errors - brief: "Number of invocation errors" - instrument: counter - unit: "{error}" - attributes: - - ref: faas.trigger - - - id: metric.faas.invocations - type: metric - metric_name: faas.invocations - brief: "Number of successful invocations" - instrument: counter - unit: "{invocation}" - attributes: - - ref: faas.trigger - - - id: metric.faas.timeouts - type: metric - metric_name: faas.timeouts - brief: "Number of invocation timeouts" - instrument: counter - unit: "{timeout}" - attributes: - - ref: faas.trigger - - - id: metric.faas.mem_usage - type: metric - metric_name: faas.mem_usage - brief: "Distribution of max memory usage per invocation" - instrument: histogram - unit: "By" - attributes: - - ref: faas.trigger - - - id: metric.faas.cpu_usage - type: metric - metric_name: faas.cpu_usage - brief: "Distribution of CPU usage per invocation" - instrument: histogram - unit: "s" - attributes: - - ref: faas.trigger - - - id: metric.faas.net_io - type: metric - metric_name: faas.net_io - brief: "Distribution of net I/O usage per invocation" - instrument: histogram - unit: "By" - attributes: - - ref: faas.trigger diff --git a/internal/otelschema/_testdata/model/metrics/http.yaml b/internal/otelschema/_testdata/model/metrics/http.yaml deleted file mode 100644 index 9e12896e..00000000 --- a/internal/otelschema/_testdata/model/metrics/http.yaml +++ /dev/null @@ -1,119 +0,0 @@ -groups: - - id: metric_attributes.http.server - type: attribute_group - brief: 'HTTP server attributes' - extends: attributes.http.server - attributes: - - ref: server.address - requirement_level: opt_in - note: | - See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). - > **Warning** - > Since this attribute is based on HTTP headers, opting in to it may allow an attacker - > to trigger cardinality limits, degrading the usefulness of the metric. - - ref: server.port - requirement_level: opt_in - note: | - See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). - > **Warning** - > Since this attribute is based on HTTP headers, opting in to it may allow an attacker - > to trigger cardinality limits, degrading the usefulness of the metric. - - id: metric_attributes.http.client - type: attribute_group - brief: 'HTTP client attributes' - extends: attributes.http.client - - - id: metric.http.server.request.duration - type: metric - metric_name: http.server.request.duration - brief: "Duration of HTTP server requests." - instrument: histogram - unit: "s" - extends: metric_attributes.http.server - - - id: metric.http.server.active_requests - type: metric - metric_name: http.server.active_requests - brief: "Number of active HTTP server requests." - instrument: updowncounter - unit: "{request}" - attributes: - - ref: http.request.method - requirement_level: required - - ref: url.scheme - requirement_level: required - examples: ["http", "https"] - - ref: server.address - requirement_level: opt_in - brief: > - Name of the local HTTP server that received the request. - note: | - See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). - > **Warning** - > Since this attribute is based on HTTP headers, opting in to it may allow an attacker - > to trigger cardinality limits, degrading the usefulness of the metric. - - ref: server.port - requirement_level: opt_in - brief: > - Port of the local HTTP server that received the request. - note: | - See [Setting `server.address` and `server.port` attributes](/docs/http/http-spans.md#setting-serveraddress-and-serverport-attributes). - > **Warning** - > Since this attribute is based on HTTP headers, opting in to it may allow an attacker - > to trigger cardinality limits, degrading the usefulness of the metric. - - - id: metric.http.server.request.body.size - type: metric - metric_name: http.server.request.body.size - brief: "Size of HTTP server request bodies." - instrument: histogram - unit: "By" - note: > - The size of the request payload body in bytes. This is the number of bytes transferred excluding headers and - is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) - header. For requests using transport encoding, this should be the compressed size. - extends: metric_attributes.http.server - - - id: metric.http.server.response.body.size - type: metric - metric_name: http.server.response.body.size - brief: "Size of HTTP server response bodies." - instrument: histogram - unit: "By" - note: > - The size of the response payload body in bytes. This is the number of bytes transferred excluding headers and - is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) - header. For requests using transport encoding, this should be the compressed size. - extends: metric_attributes.http.server - - - id: metric.http.client.request.duration - type: metric - metric_name: http.client.request.duration - brief: "Duration of HTTP client requests." - instrument: histogram - unit: "s" - extends: metric_attributes.http.client - - - id: metric.http.client.request.body.size - type: metric - metric_name: http.client.request.body.size - brief: "Size of HTTP client request bodies." - instrument: histogram - unit: "By" - note: > - The size of the request payload body in bytes. This is the number of bytes transferred excluding headers and - is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) - header. For requests using transport encoding, this should be the compressed size. - extends: metric_attributes.http.client - - - id: metric.http.client.response.body.size - type: metric - metric_name: http.client.response.body.size - brief: "Size of HTTP client response bodies." - instrument: histogram - unit: "By" - note: > - The size of the response payload body in bytes. This is the number of bytes transferred excluding headers and - is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) - header. For requests using transport encoding, this should be the compressed size. - extends: metric_attributes.http.client diff --git a/internal/otelschema/_testdata/model/metrics/jvm-metrics-experimental.yaml b/internal/otelschema/_testdata/model/metrics/jvm-metrics-experimental.yaml deleted file mode 100644 index 8ea4fa57..00000000 --- a/internal/otelschema/_testdata/model/metrics/jvm-metrics-experimental.yaml +++ /dev/null @@ -1,70 +0,0 @@ -groups: - - id: metric.jvm.memory.init - type: metric - metric_name: jvm.memory.init - extends: attributes.jvm.memory - brief: "Measure of initial memory requested." - instrument: updowncounter - unit: "By" - - - id: metric.jvm.system.cpu.utilization - type: metric - metric_name: jvm.system.cpu.utilization - brief: "Recent CPU utilization for the whole system as reported by the JVM." - note: > - The value range is [0.0,1.0]. - This utilization is not defined as being for the specific interval since last measurement - (unlike `system.cpu.utilization`). - [Reference](https://docs.oracle.com/en/java/javase/17/docs/api/jdk.management/com/sun/management/OperatingSystemMXBean.html#getCpuLoad()). - instrument: gauge - unit: "1" - - - id: metric.jvm.system.cpu.load_1m - type: metric - metric_name: jvm.system.cpu.load_1m - brief: "Average CPU load of the whole system for the last minute as reported by the JVM." - note: > - The value range is [0,n], where n is the number of CPU cores - or a negative number if the value is not available. - This utilization is not defined as being for the specific interval since last measurement - (unlike `system.cpu.utilization`). - [Reference](https://docs.oracle.com/en/java/javase/17/docs/api/java.management/java/lang/management/OperatingSystemMXBean.html#getSystemLoadAverage()). - instrument: gauge - unit: "{run_queue_item}" - - - id: attributes.jvm.buffer - type: attribute_group - brief: "Describes JVM buffer metric attributes." - prefix: jvm.buffer - attributes: - - id: pool.name - type: string - requirement_level: recommended - brief: Name of the buffer pool. - examples: [ "mapped", "direct" ] - note: > - Pool names are generally obtained via - [BufferPoolMXBean#getName()](https://docs.oracle.com/en/java/javase/11/docs/api/java.management/java/lang/management/BufferPoolMXBean.html#getName()). - - - id: metric.jvm.buffer.memory.usage - type: metric - metric_name: jvm.buffer.memory.usage - extends: attributes.jvm.buffer - brief: "Measure of memory used by buffers." - instrument: updowncounter - unit: "By" - - - id: metric.jvm.buffer.memory.limit - type: metric - metric_name: jvm.buffer.memory.limit - extends: attributes.jvm.buffer - brief: "Measure of total memory capacity of buffers." - instrument: updowncounter - unit: "By" - - - id: metric.jvm.buffer.count - type: metric - metric_name: jvm.buffer.count - extends: attributes.jvm.buffer - brief: "Number of buffers in the pool." - instrument: updowncounter - unit: "{buffer}" diff --git a/internal/otelschema/_testdata/model/metrics/jvm-metrics.yaml b/internal/otelschema/_testdata/model/metrics/jvm-metrics.yaml deleted file mode 100644 index ba1f5604..00000000 --- a/internal/otelschema/_testdata/model/metrics/jvm-metrics.yaml +++ /dev/null @@ -1,168 +0,0 @@ -groups: - - id: attributes.jvm.memory - type: attribute_group - brief: "Describes JVM memory metric attributes." - prefix: jvm.memory - attributes: - - id: type - type: - allow_custom_values: false - members: - - id: heap - value: 'heap' - brief: 'Heap memory.' - - id: non_heap - value: 'non_heap' - brief: 'Non-heap memory' - requirement_level: recommended - brief: The type of memory. - examples: ["heap", "non_heap"] - - id: pool.name - type: string - requirement_level: recommended - brief: Name of the memory pool. - examples: ["G1 Old Gen", "G1 Eden space", "G1 Survivor Space"] - note: > - Pool names are generally obtained via - [MemoryPoolMXBean#getName()](https://docs.oracle.com/en/java/javase/11/docs/api/java.management/java/lang/management/MemoryPoolMXBean.html#getName()). - - - id: metric.jvm.memory.used - type: metric - metric_name: jvm.memory.used - extends: attributes.jvm.memory - brief: "Measure of memory used." - instrument: updowncounter - unit: "By" - - - id: metric.jvm.memory.committed - type: metric - metric_name: jvm.memory.committed - extends: attributes.jvm.memory - brief: "Measure of memory committed." - instrument: updowncounter - unit: "By" - - - id: metric.jvm.memory.limit - type: metric - metric_name: jvm.memory.limit - extends: attributes.jvm.memory - brief: "Measure of max obtainable memory." - instrument: updowncounter - unit: "By" - - - id: metric.jvm.memory.used_after_last_gc - type: metric - metric_name: jvm.memory.used_after_last_gc - extends: attributes.jvm.memory - brief: "Measure of memory used, as measured after the most recent garbage collection event on this pool." - instrument: updowncounter - unit: "By" - - - id: metric.jvm.gc.duration - type: metric - metric_name: jvm.gc.duration - brief: "Duration of JVM garbage collection actions." - instrument: histogram - unit: "s" - prefix: jvm.gc - attributes: - - id: name - type: string - requirement_level: recommended - brief: Name of the garbage collector. - examples: ["G1 Young Generation", "G1 Old Generation"] - note: > - Garbage collector name is generally obtained via - [GarbageCollectionNotificationInfo#getGcName()](https://docs.oracle.com/en/java/javase/11/docs/api/jdk.management/com/sun/management/GarbageCollectionNotificationInfo.html#getGcName()). - - id: action - type: string - requirement_level: recommended - brief: Name of the garbage collector action. - examples: ["end of minor GC", "end of major GC"] - note: > - Garbage collector action is generally obtained via - [GarbageCollectionNotificationInfo#getGcAction()](https://docs.oracle.com/en/java/javase/11/docs/api/jdk.management/com/sun/management/GarbageCollectionNotificationInfo.html#getGcAction()). - - - id: metric.jvm.thread.count - type: metric - metric_name: jvm.thread.count - brief: "Number of executing platform threads." - instrument: updowncounter - unit: "{thread}" - attributes: - - id: jvm.thread.daemon - type: boolean - requirement_level: recommended - brief: "Whether the thread is daemon or not." - - id: jvm.thread.state - requirement_level: recommended - type: - allow_custom_values: false - members: - - id: new - value: 'new' - brief: 'A thread that has not yet started is in this state.' - - id: runnable - value: 'runnable' - brief: 'A thread executing in the Java virtual machine is in this state.' - - id: blocked - value: 'blocked' - brief: 'A thread that is blocked waiting for a monitor lock is in this state.' - - id: waiting - value: 'waiting' - brief: 'A thread that is waiting indefinitely for another thread to perform a particular action is in this state.' - - id: timed_waiting - value: 'timed_waiting' - brief: 'A thread that is waiting for another thread to perform an action for up to a specified waiting time is in this state.' - - id: terminated - value: 'terminated' - brief: 'A thread that has exited is in this state.' - brief: "State of the thread." - examples: ["runnable", "blocked"] - - - id: metric.jvm.class.loaded - type: metric - metric_name: jvm.class.loaded - brief: "Number of classes loaded since JVM start." - instrument: counter - unit: "{class}" - - - id: metric.jvm.class.unloaded - type: metric - metric_name: jvm.class.unloaded - brief: "Number of classes unloaded since JVM start." - instrument: counter - unit: "{class}" - - - id: metric.jvm.class.count - type: metric - metric_name: jvm.class.count - brief: "Number of classes currently loaded." - instrument: updowncounter - unit: "{class}" - - - id: metric.jvm.cpu.count - type: metric - metric_name: jvm.cpu.count - brief: "Number of processors available to the Java virtual machine." - instrument: updowncounter - unit: "{cpu}" - - - id: metric.jvm.cpu.time - type: metric - metric_name: jvm.cpu.time - brief: "CPU time used by the process as reported by the JVM." - instrument: counter - unit: "s" - - - id: metric.jvm.cpu.recent_utilization - type: metric - metric_name: jvm.cpu.recent_utilization - brief: "Recent CPU utilization for the process as reported by the JVM." - note: > - The value range is [0.0,1.0]. - This utilization is not defined as being for the specific interval since last measurement - (unlike `system.cpu.utilization`). - [Reference](https://docs.oracle.com/en/java/javase/17/docs/api/jdk.management/com/sun/management/OperatingSystemMXBean.html#getProcessCpuLoad()). - instrument: gauge - unit: "1" diff --git a/internal/otelschema/_testdata/model/metrics/rpc-metrics.yaml b/internal/otelschema/_testdata/model/metrics/rpc-metrics.yaml deleted file mode 100644 index ca29cdf1..00000000 --- a/internal/otelschema/_testdata/model/metrics/rpc-metrics.yaml +++ /dev/null @@ -1,122 +0,0 @@ -groups: - # TODO: Should we list the attributes on each metric - # OR leave a single table like it is today? Since all attributes are applied - # to all metrics, the repetition of them bloats the page - - id: attributes.metrics.rpc - type: attribute_group - brief: "Describes RPC metric attributes." - attributes: - - ref: rpc.system - requirement_level: required - - ref: rpc.service - - ref: rpc.method - - ref: network.transport - - ref: network.type - - ref: server.address - - ref: server.port - - # RPC Server metrics - - id: metric.rpc.server.duration - type: metric - metric_name: rpc.server.duration - brief: Measures the duration of inbound RPC. - instrument: histogram - unit: "ms" - note: | - While streaming RPCs may record this metric as start-of-batch - to end-of-batch, it's hard to interpret in practice. - - **Streaming**: N/A. - - - id: metric.rpc.server.request.size - type: metric - metric_name: rpc.server.request.size - brief: Measures the size of RPC request messages (uncompressed). - instrument: histogram - unit: "By" - note: | - **Streaming**: Recorded per message in a streaming batch - - - id: metric.rpc.server.response.size - type: metric - metric_name: rpc.server.response.size - brief: Measures the size of RPC response messages (uncompressed). - instrument: histogram - unit: "By" - note: | - **Streaming**: Recorded per response in a streaming batch - - - id: metric.rpc.server.requests_per_rpc - type: metric - metric_name: rpc.server.requests_per_rpc - brief: Measures the number of messages received per RPC. - instrument: histogram - unit: "{count}" - note: | - Should be 1 for all non-streaming RPCs. - - **Streaming** : This metric is required for server and client streaming RPCs - - - id: metric.rpc.server.responses_per_rpc - type: metric - metric_name: rpc.server.responses_per_rpc - brief: Measures the number of messages sent per RPC. - instrument: histogram - unit: "{count}" - note: | - Should be 1 for all non-streaming RPCs. - - **Streaming**: This metric is required for server and client streaming RPCs - - # RPC Client metrics - - id: metric.rpc.client.duration - type: metric - metric_name: rpc.client.duration - brief: Measures the duration of outbound RPC. - instrument: histogram - unit: "ms" - note: | - While streaming RPCs may record this metric as start-of-batch - to end-of-batch, it's hard to interpret in practice. - - **Streaming**: N/A. - - - id: metric.rpc.client.request.size - type: metric - metric_name: rpc.client.request.size - brief: Measures the size of RPC request messages (uncompressed). - instrument: histogram - unit: "By" - note: | - **Streaming**: Recorded per message in a streaming batch - - - id: metric.rpc.client.response.size - type: metric - metric_name: rpc.client.response.size - brief: Measures the size of RPC response messages (uncompressed). - instrument: histogram - unit: "By" - note: | - **Streaming**: Recorded per response in a streaming batch - - - id: metric.rpc.client.requests_per_rpc - type: metric - metric_name: rpc.client.requests_per_rpc - brief: Measures the number of messages received per RPC. - instrument: histogram - unit: "{count}" - note: | - Should be 1 for all non-streaming RPCs. - - **Streaming**: This metric is required for server and client streaming RPCs - - - id: metric.rpc.client.responses_per_rpc - type: metric - metric_name: rpc.client.responses_per_rpc - brief: Measures the number of messages sent per RPC. - instrument: histogram - unit: "{count}" - note: | - Should be 1 for all non-streaming RPCs. - - **Streaming**: This metric is required for server and client streaming RPCs diff --git a/internal/otelschema/_testdata/model/metrics/system-metrics.yaml b/internal/otelschema/_testdata/model/metrics/system-metrics.yaml deleted file mode 100644 index d411115e..00000000 --- a/internal/otelschema/_testdata/model/metrics/system-metrics.yaml +++ /dev/null @@ -1,501 +0,0 @@ -groups: - # General system attributes - - id: attributes.system - prefix: system - type: attribute_group - brief: "Describes System metric attributes" - attributes: - - id: device - type: string - brief: "The device identifier" - examples: ["(identifier)"] - - # system.cpu.* metrics and attribute group - - id: attributes.system.cpu - prefix: system.cpu - type: attribute_group - brief: "Describes System CPU metric attributes" - attributes: - - id: state - type: - allow_custom_values: true - members: - - id: user - value: 'user' - - id: system - value: 'system' - - id: nice - value: 'nice' - - id: idle - value: 'idle' - - id: iowait - value: 'iowait' - - id: interrupt - value: 'interrupt' - - id: steal - value: 'steal' - brief: "The state of the CPU" - examples: ["idle", "interrupt"] - - id: logical_number - type: int - brief: "The logical CPU number [0..n-1]" - examples: [1] - - - id: metric.system.cpu.time - type: metric - metric_name: system.cpu.time - brief: "Seconds each logical CPU spent on each mode" - instrument: counter - unit: "s" - attributes: - - ref: system.cpu.state - - ref: system.cpu.logical_number - - - id: metric.system.cpu.utilization - type: metric - metric_name: system.cpu.utilization - brief: "Difference in system.cpu.time since the last measurement, divided by the elapsed time and number of logical CPUs" - instrument: gauge - unit: "1" - attributes: - - ref: system.cpu.state - - ref: system.cpu.logical_number - - - id: metric.system.cpu.frequency - type: metric - metric_name: system.cpu.frequency - brief: "Reports the current frequency of the CPU in Hz" - instrument: gauge - unit: "{Hz}" - attributes: - - ref: system.cpu.logical_number - - - id: metric.system.cpu.physical.count - type: metric - metric_name: system.cpu.physical.count - brief: "Reports the number of actual physical processor cores on the hardware" - instrument: updowncounter - unit: "{cpu}" - attributes: [] - - - id: metric.system.cpu.logical.count - type: metric - metric_name: system.cpu.logical.count - brief: "Reports the number of logical (virtual) processor cores created by the operating system to manage multitasking" - instrument: updowncounter - unit: "{cpu}" - attributes: [] - - # sytem.memory.* metrics and attribute group - - id: attributes.system.memory - prefix: system.memory - type: attribute_group - brief: "Describes System Memory metric attributes" - attributes: - - id: state - type: - allow_custom_values: true - members: - - id: used - value: 'used' - - id: free - value: 'free' - - id: shared - value: 'shared' - - id: buffers - value: 'buffers' - - id: cached - value: 'cached' - brief: "The memory state" - examples: ["free", "cached"] - - - id: metric.system.memory.usage - type: metric - metric_name: system.memory.usage - brief: "Reports memory in use by state." - note: | - The sum over all `system.memory.state` values SHOULD equal the total memory - available on the system, that is `system.memory.limit`. - instrument: updowncounter - unit: "By" - attributes: - - ref: system.memory.state - - - id: metric.system.memory.limit - type: metric - metric_name: system.memory.limit - brief: "Total memory available in the system." - note: | - Its value SHOULD equal the sum of `system.memory.state` over all states. - instrument: updowncounter - unit: "By" - - - id: metric.system.memory.utilization - type: metric - metric_name: system.memory.utilization - brief: "" - instrument: gauge - unit: "1" - attributes: - - ref: system.memory.state - - # system.paging.* metrics and attribute group - - id: attributes.system.paging - prefix: system.paging - type: attribute_group - brief: "Describes System Memory Paging metric attributes" - attributes: - - id: state - type: - allow_custom_values: false - members: - - id: used - value: 'used' - - id: free - value: 'free' - brief: "The memory paging state" - examples: ["free"] - - id: type - type: - allow_custom_values: false - members: - - id: major - value: 'major' - - id: minor - value: 'minor' - brief: "The memory paging type" - examples: ["minor"] - - id: direction - type: - allow_custom_values: false - members: - - id: in - value: 'in' - - id: out - value: 'out' - brief: "The paging access direction" - examples: ["in"] - - id: metric.system.paging.usage - type: metric - metric_name: system.paging.usage - brief: "Unix swap or windows pagefile usage" - instrument: updowncounter - unit: "By" - attributes: - - ref: system.paging.state - - - id: metric.system.paging.utilization - type: metric - metric_name: system.paging.utilization - brief: "" - instrument: gauge - unit: "1" - attributes: - - ref: system.paging.state - - - id: metric.system.paging.faults - type: metric - metric_name: system.paging.faults - brief: "" - instrument: counter - unit: "{fault}" - attributes: - - ref: system.paging.type - - - id: metric.system.paging.operations - type: metric - metric_name: system.paging.operations - brief: "" - instrument: counter - unit: "{operation}" - attributes: - - ref: system.paging.type - - ref: system.paging.direction - - # system.disk.* metrics and attribute group - - id: metric.system.disk.io - type: metric - metric_name: system.disk.io - brief: "" - instrument: counter - unit: "By" - attributes: - - ref: system.device - - ref: disk.io.direction - - - id: metric.system.disk.operations - type: metric - metric_name: system.disk.operations - brief: "" - instrument: counter - unit: "{operation}" - attributes: - - ref: system.device - - ref: disk.io.direction - - - id: metric.system.disk.io_time - type: metric - metric_name: system.disk.io_time - brief: "Time disk spent activated" - instrument: counter - unit: "s" - note: | - The real elapsed time ("wall clock") used in the I/O path (time from operations running in parallel are not counted). Measured as: - - - Linux: Field 13 from [procfs-diskstats](https://www.kernel.org/doc/Documentation/ABI/testing/procfs-diskstats) - - Windows: The complement of - ["Disk\% Idle Time"](https://learn.microsoft.com/archive/blogs/askcore/windows-performance-monitor-disk-counters-explained#windows-performance-monitor-disk-counters-explained) - performance counter: `uptime * (100 - "Disk\% Idle Time") / 100` - attributes: - - ref: system.device - - - id: metric.system.disk.operation_time - type: metric - metric_name: system.disk.operation_time - brief: "Sum of the time each operation took to complete" - instrument: counter - unit: "s" - note: | - Because it is the sum of time each request took, parallel-issued requests each contribute to make the count grow. Measured as: - - - Linux: Fields 7 & 11 from [procfs-diskstats](https://www.kernel.org/doc/Documentation/ABI/testing/procfs-diskstats) - - Windows: "Avg. Disk sec/Read" perf counter multiplied by "Disk Reads/sec" perf counter (similar for Writes) - attributes: - - ref: system.device - - ref: disk.io.direction - - - id: metric.system.disk.merged - type: metric - metric_name: system.disk.merged - brief: "" - instrument: counter - unit: "{operation}" - attributes: - - ref: system.device - - ref: disk.io.direction - - # system.filesystem.* metrics and attribute group - - id: attributes.system.filesystem - prefix: system.filesystem - type: attribute_group - brief: "Describes Filesystem metric attributes" - attributes: - - id: state - brief: "The filesystem state" - type: - allow_custom_values: false - members: - - id: used - value: 'used' - - id: free - value: 'free' - - id: reserved - value: 'reserved' - examples: ["used"] - - id: type - type: - allow_custom_values: true - members: - - id: fat32 - value: 'fat32' - - id: exfat - value: 'exfat' - - id: ntfs - value: 'ntfs' - - id: refs - value: 'refs' - - id: hfsplus - value: 'hfsplus' - - id: ext4 - value: 'ext4' - brief: "The filesystem type" - examples: ["ext4"] - - id: mode - type: string - brief: "The filesystem mode" - examples: ["rw, ro"] - - id: mountpoint - type: string - brief: "The filesystem mount path" - examples: ["/mnt/data"] - - - id: metric.system.filesystem.usage - type: metric - metric_name: system.filesystem.usage - brief: "" - instrument: updowncounter - unit: "By" - attributes: - - ref: system.device - - ref: system.filesystem.state - - ref: system.filesystem.type - - ref: system.filesystem.mode - - ref: system.filesystem.mountpoint - - - id: metric.system.filesystem.utilization - type: metric - metric_name: system.filesystem.utilization - brief: "" - instrument: gauge - unit: "1" - attributes: - - ref: system.device - - ref: system.filesystem.state - - ref: system.filesystem.type - - ref: system.filesystem.mode - - ref: system.filesystem.mountpoint - - # system.network.* metrics and attribute group - - # System-specific network attributes - - id: attributes.system.network - prefix: system.network - type: attribute_group - brief: "Describes Network metric attributes" - attributes: - - id: state - type: - allow_custom_values: false - members: - - id: close - value: 'close' - - id: close_wait - value: 'close_wait' - - id: closing - value: 'closing' - - id: delete - value: 'delete' - - id: established - value: 'established' - - id: fin_wait_1 - value: 'fin_wait_1' - - id: fin_wait_2 - value: 'fin_wait_2' - - id: last_ack - value: 'last_ack' - - id: listen - value: 'listen' - - id: syn_recv - value: 'syn_recv' - - id: syn_sent - value: 'syn_sent' - - id: time_wait - value: 'time_wait' - brief: "A stateless protocol MUST NOT set this attribute" - examples: ["close_wait"] - - - id: metric.system.network.dropped - type: metric - metric_name: system.network.dropped - brief: "Count of packets that are dropped or discarded even though there was no error" - instrument: counter - unit: "{packet}" - note: | - Measured as: - - - Linux: the `drop` column in `/proc/dev/net` ([source](https://web.archive.org/web/20180321091318/http://www.onlamp.com/pub/a/linux/2000/11/16/LinuxAdmin.html)) - - Windows: [`InDiscards`/`OutDiscards`](https://docs.microsoft.com/windows/win32/api/netioapi/ns-netioapi-mib_if_row2) - from [`GetIfEntry2`](https://docs.microsoft.com/windows/win32/api/netioapi/nf-netioapi-getifentry2) - attributes: - - ref: system.device - - ref: network.io.direction - - - id: metric.system.network.packets - type: metric - metric_name: system.network.packets - brief: "" - instrument: counter - unit: "{packet}" - attributes: - - ref: system.device - - ref: network.io.direction - - - id: metric.system.network.errors - type: metric - metric_name: system.network.errors - brief: "Count of network errors detected" - instrument: counter - unit: "{error}" - note: | - Measured as: - - - Linux: the `errs` column in `/proc/dev/net` ([source](https://web.archive.org/web/20180321091318/http://www.onlamp.com/pub/a/linux/2000/11/16/LinuxAdmin.html)). - - Windows: [`InErrors`/`OutErrors`](https://docs.microsoft.com/windows/win32/api/netioapi/ns-netioapi-mib_if_row2) - from [`GetIfEntry2`](https://docs.microsoft.com/windows/win32/api/netioapi/nf-netioapi-getifentry2). - attributes: - - ref: system.device - - ref: network.io.direction - - - id: metric.system.network.io - type: metric - metric_name: system.network.io - brief: "" - instrument: counter - unit: "By" - attributes: - - ref: system.device - - ref: network.io.direction - - - id: metric.system.network.connections - type: metric - metric_name: system.network.connections - brief: "" - instrument: updowncounter - unit: "{connection}" - attributes: - - ref: system.device - - ref: system.network.state - - ref: network.transport - - # system.processes.* metrics and attribute group - - id: attributes.system.processes - prefix: system.processes - type: attribute_group - brief: "Describes System Process metric attributes" - attributes: - - id: status - type: - allow_custom_values: true - members: - - id: running - value: 'running' - - id: sleeping - value: 'sleeping' - - id: stopped - value: 'stopped' - - id: defunct - value: 'defunct' - brief: > - The process state, e.g., [Linux Process State Codes](https://man7.org/linux/man-pages/man1/ps.1.html#PROCESS_STATE_CODES) - examples: ["running"] - - - - id: metric.system.processes.count - type: metric - metric_name: system.processes.count - brief: "Total number of processes in each state" - instrument: updowncounter - unit: "{process}" - attributes: - - ref: system.processes.status - - - id: metric.system.processes.created - type: metric - metric_name: system.processes.created - brief: "Total number of processes created over uptime of the host" - instrument: counter - unit: "{process}" - - # system.linux.* metrics - - id: metric.system.linux.memory.available - type: metric - metric_name: system.linux.memory.available - brief: "An estimate of how much memory is available for starting new applications, without causing swapping" - note: | - This is an alternative to `system.memory.usage` metric with `state=free`. - Linux starting from 3.14 exports "available" memory. It takes "free" memory as a baseline, and then factors in kernel-specific values. - This is supposed to be more accurate than just "free" memory. - For reference, see the calculations [here](https://superuser.com/a/980821). - See also `MemAvailable` in [/proc/meminfo](https://man7.org/linux/man-pages/man5/proc.5.html). - instrument: updowncounter - unit: "By" diff --git a/internal/otelschema/_testdata/model/network.yaml b/internal/otelschema/_testdata/model/network.yaml deleted file mode 100644 index 7e1ee396..00000000 --- a/internal/otelschema/_testdata/model/network.yaml +++ /dev/null @@ -1,28 +0,0 @@ -groups: - - id: network-core - prefix: network - type: attribute_group - brief: > - These attributes may be used for any network related operation. - attributes: - - ref: network.transport - - ref: network.type - - ref: network.protocol.name - - ref: network.protocol.version - - ref: network.peer.address - - ref: network.peer.port - - ref: network.local.address - - ref: network.local.port - - - id: network-connection-and-carrier - prefix: network - type: attribute_group - brief: > - These attributes may be used for any network related operation. - attributes: - - ref: network.connection.type - - ref: network.connection.subtype - - ref: network.carrier.name - - ref: network.carrier.mcc - - ref: network.carrier.mnc - - ref: network.carrier.icc diff --git a/internal/otelschema/_testdata/model/registry/client.yaml b/internal/otelschema/_testdata/model/registry/client.yaml deleted file mode 100644 index bd214a17..00000000 --- a/internal/otelschema/_testdata/model/registry/client.yaml +++ /dev/null @@ -1,28 +0,0 @@ -groups: - - id: client - prefix: client - type: attribute_group - brief: > - These attributes may be used to describe the client in a connection-based network interaction - where there is one side that initiates the connection (the client is the side that initiates the connection). - This covers all TCP network interactions since TCP is connection-based and one side initiates the - connection (an exception is made for peer-to-peer communication over TCP where the "user-facing" surface of the - protocol / API doesn't expose a clear notion of client and server). - This also covers UDP network interactions where one side initiates the interaction, e.g. QUIC (HTTP/3) and DNS. - attributes: - - id: address - stability: stable - type: string - brief: "Client address - domain name if available without reverse DNS lookup; otherwise, IP address or Unix domain socket name." - note: > - When observed from the server side, and when communicating through an intermediary, `client.address` SHOULD represent - the client address behind any intermediaries, for example proxies, if it's available. - examples: ['client.example.com', '10.1.2.80', '/tmp/my.sock'] - - id: port - stability: stable - type: int - brief: Client port number. - examples: [65123] - note: > - When observed from the server side, and when communicating through an intermediary, `client.port` SHOULD represent - the client port behind any intermediaries, for example proxies, if it's available. diff --git a/internal/otelschema/_testdata/model/registry/cloud.yaml b/internal/otelschema/_testdata/model/registry/cloud.yaml deleted file mode 100644 index 2c669052..00000000 --- a/internal/otelschema/_testdata/model/registry/cloud.yaml +++ /dev/null @@ -1,179 +0,0 @@ -groups: - - id: registry.cloud - prefix: cloud - type: resource - brief: > - A cloud environment (e.g. GCP, Azure, AWS). - attributes: - - id: provider - type: - allow_custom_values: true - members: - - id: 'alibaba_cloud' - value: 'alibaba_cloud' - brief: 'Alibaba Cloud' - - id: 'aws' - value: 'aws' - brief: 'Amazon Web Services' - - id: 'azure' - value: 'azure' - brief: 'Microsoft Azure' - - id: 'gcp' - value: 'gcp' - brief: 'Google Cloud Platform' - - id: 'heroku' - value: 'heroku' - brief: 'Heroku Platform as a Service' - - id: 'ibm_cloud' - value: 'ibm_cloud' - brief: 'IBM Cloud' - - id: 'tencent_cloud' - value: 'tencent_cloud' - brief: 'Tencent Cloud' - - brief: > - Name of the cloud provider. - - id: account.id - type: string - brief: > - The cloud account ID the resource is assigned to. - examples: ['111111111111', 'opentelemetry'] - - id: region - type: string - brief: > - The geographical region the resource is running. - note: > - Refer to your provider's docs to see the available regions, for example - [Alibaba Cloud regions](https://www.alibabacloud.com/help/doc-detail/40654.htm), - [AWS regions](https://aws.amazon.com/about-aws/global-infrastructure/regions_az/), - [Azure regions](https://azure.microsoft.com/global-infrastructure/geographies/), - [Google Cloud regions](https://cloud.google.com/about/locations), - or [Tencent Cloud regions](https://www.tencentcloud.com/document/product/213/6091). - examples: ['us-central1', 'us-east-1'] - - id: resource_id - type: string - brief: > - Cloud provider-specific native identifier of the monitored cloud resource - (e.g. an [ARN](https://docs.aws.amazon.com/general/latest/gr/aws-arns-and-namespaces.html) on AWS, - a [fully qualified resource ID](https://learn.microsoft.com/rest/api/resources/resources/get-by-id) on Azure, - a [full resource name](https://cloud.google.com/apis/design/resource_names#full_resource_name) on GCP) - note: | - On some cloud providers, it may not be possible to determine the full ID at startup, - so it may be necessary to set `cloud.resource_id` as a span attribute instead. - - The exact value to use for `cloud.resource_id` depends on the cloud provider. - The following well-known definitions MUST be used if you set this attribute and they apply: - - * **AWS Lambda:** The function [ARN](https://docs.aws.amazon.com/general/latest/gr/aws-arns-and-namespaces.html). - Take care not to use the "invoked ARN" directly but replace any - [alias suffix](https://docs.aws.amazon.com/lambda/latest/dg/configuration-aliases.html) - with the resolved function version, as the same runtime instance may be invokable with - multiple different aliases. - * **GCP:** The [URI of the resource](https://cloud.google.com/iam/docs/full-resource-names) - * **Azure:** The [Fully Qualified Resource ID](https://docs.microsoft.com/rest/api/resources/resources/get-by-id) of the invoked function, - *not* the function app, having the form - `/subscriptions//resourceGroups//providers/Microsoft.Web/sites//functions/`. - This means that a span attribute MUST be used, as an Azure function app can host multiple functions that would usually share - a TracerProvider. - examples: - - 'arn:aws:lambda:REGION:ACCOUNT_ID:function:my-function' - - '//run.googleapis.com/projects/PROJECT_ID/locations/LOCATION_ID/services/SERVICE_ID' - - '/subscriptions//resourceGroups//providers/Microsoft.Web/sites//functions/' - - id: availability_zone - type: string - brief: > - Cloud regions often have multiple, isolated locations known as zones - to increase availability. Availability zone represents the - zone where the resource is running. - note: > - Availability zones are called "zones" on Alibaba Cloud and Google Cloud. - examples: ['us-east-1c'] - - id: platform - type: - allow_custom_values: true - members: - - id: alibaba_cloud_ecs - value: 'alibaba_cloud_ecs' - brief: Alibaba Cloud Elastic Compute Service - - id: alibaba_cloud_fc - value: 'alibaba_cloud_fc' - brief: Alibaba Cloud Function Compute - - id: alibaba_cloud_openshift - value: 'alibaba_cloud_openshift' - brief: Red Hat OpenShift on Alibaba Cloud - - id: aws_ec2 - value: 'aws_ec2' - brief: AWS Elastic Compute Cloud - - id: aws_ecs - value: 'aws_ecs' - brief: AWS Elastic Container Service - - id: aws_eks - value: 'aws_eks' - brief: AWS Elastic Kubernetes Service - - id: aws_lambda - value: 'aws_lambda' - brief: AWS Lambda - - id: aws_elastic_beanstalk - value: 'aws_elastic_beanstalk' - brief: AWS Elastic Beanstalk - - id: aws_app_runner - value: 'aws_app_runner' - brief: AWS App Runner - - id: aws_openshift - value: 'aws_openshift' - brief: Red Hat OpenShift on AWS (ROSA) - - id: azure_vm - value: 'azure_vm' - brief: Azure Virtual Machines - - id: azure_container_instances - value: 'azure_container_instances' - brief: Azure Container Instances - - id: azure_aks - value: 'azure_aks' - brief: Azure Kubernetes Service - - id: azure_functions - value: 'azure_functions' - brief: Azure Functions - - id: azure_app_service - value: 'azure_app_service' - brief: Azure App Service - - id: azure_openshift - value: 'azure_openshift' - brief: Azure Red Hat OpenShift - - id: gcp_bare_metal_solution - value: 'gcp_bare_metal_solution' - brief: Google Bare Metal Solution (BMS) - - id: gcp_compute_engine - value: 'gcp_compute_engine' - brief: Google Cloud Compute Engine (GCE) - - id: gcp_cloud_run - value: 'gcp_cloud_run' - brief: Google Cloud Run - - id: gcp_kubernetes_engine - value: 'gcp_kubernetes_engine' - brief: Google Cloud Kubernetes Engine (GKE) - - id: gcp_cloud_functions - value: 'gcp_cloud_functions' - brief: Google Cloud Functions (GCF) - - id: gcp_app_engine - value: 'gcp_app_engine' - brief: Google Cloud App Engine (GAE) - - id: gcp_openshift - value: 'gcp_openshift' - brief: Red Hat OpenShift on Google Cloud - - id: ibm_cloud_openshift - value: 'ibm_cloud_openshift' - brief: Red Hat OpenShift on IBM Cloud - - id: tencent_cloud_cvm - value: 'tencent_cloud_cvm' - brief: Tencent Cloud Cloud Virtual Machine (CVM) - - id: tencent_cloud_eks - value: 'tencent_cloud_eks' - brief: Tencent Cloud Elastic Kubernetes Service (EKS) - - id: tencent_cloud_scf - value: 'tencent_cloud_scf' - brief: Tencent Cloud Serverless Cloud Function (SCF) - brief: > - The cloud platform in use. - note: > - The prefix of the service SHOULD match the one specified in `cloud.provider`. diff --git a/internal/otelschema/_testdata/model/registry/code.yaml b/internal/otelschema/_testdata/model/registry/code.yaml deleted file mode 100644 index d5afceea..00000000 --- a/internal/otelschema/_testdata/model/registry/code.yaml +++ /dev/null @@ -1,41 +0,0 @@ -groups: - - id: registry.code - prefix: code - type: span - brief: > - These attributes allow to report this unit of code and therefore to provide more context about the span. - attributes: - - id: function - type: string - brief: > - The method or function name, or equivalent (usually rightmost part of the code unit's name). - examples: serveRequest - - id: namespace - type: string - brief: > - The "namespace" within which `code.function` is defined. Usually the qualified class or module name, - such that `code.namespace` + some separator + `code.function` form a unique identifier for the code unit. - examples: com.example.MyHttpService - - id: filepath - type: string - brief: > - The source code file name that identifies the code unit as uniquely as possible (preferably an absolute file path). - examples: /usr/local/MyApplication/content_root/app/index.php - - id: lineno - type: int - brief: > - The line number in `code.filepath` best representing the operation. It SHOULD point within the code unit named in `code.function`. - examples: 42 - - id: column - type: int - brief: > - The column number in `code.filepath` best representing the operation. It SHOULD point within the code unit named in `code.function`. - examples: 16 - - id: stacktrace - type: string - brief: > - A stacktrace as a string in the natural representation for the language runtime. - The representation is to be determined and documented by each language SIG. - examples: 'at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n - at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n - at com.example.GenerateTrace.main(GenerateTrace.java:5)' diff --git a/internal/otelschema/_testdata/model/registry/container.yaml b/internal/otelschema/_testdata/model/registry/container.yaml deleted file mode 100644 index 0755d078..00000000 --- a/internal/otelschema/_testdata/model/registry/container.yaml +++ /dev/null @@ -1,86 +0,0 @@ -groups: - - id: registry.container - prefix: container - type: resource - brief: > - A container instance. - attributes: - - id: name - type: string - brief: > - Container name used by container runtime. - examples: ['opentelemetry-autoconf'] - - id: id - type: string - brief: > - Container ID. Usually a UUID, as for example used to - [identify Docker containers](https://docs.docker.com/engine/reference/run/#container-identification). - The UUID might be abbreviated. - examples: ['a3bf90e006b2'] - - id: runtime - type: string - brief: > - The container runtime managing this container. - examples: ['docker', 'containerd', 'rkt'] - - id: image.name - type: string - brief: > - Name of the image the container was built on. - examples: ['gcr.io/opentelemetry/operator'] - - id: image.tags - type: string[] - brief: > - Container image tags. An example can be found in - [Docker Image Inspect](https://docs.docker.com/engine/api/v1.43/#tag/Image/operation/ImageInspect). - Should be only the `` section of the full name for example - from `registry.example.com/my-org/my-image:`. - examples: ['v1.27.1', '3.5.7-0'] - - id: image.id - type: string - brief: > - Runtime specific image identifier. Usually a hash algorithm followed by a UUID. - note: > - Docker defines a sha256 of the image id; `container.image.id` corresponds to the `Image` field from the Docker - container inspect [API](https://docs.docker.com/engine/api/v1.43/#tag/Container/operation/ContainerInspect) - endpoint. - - K8s defines a link to the container registry repository with digest `"imageID": "registry.azurecr.io - /namespace/service/dockerfile@sha256:bdeabd40c3a8a492eaf9e8e44d0ebbb84bac7ee25ac0cf8a7159d25f62555625"`. - - The ID is assinged by the container runtime and can vary in different environments. - Consider using `oci.manifest.digest` if it is important to identify the same - image in different environments/runtimes. - examples: ['sha256:19c92d0a00d1b66d897bceaa7319bee0dd38a10a851c60bcec9474aa3f01e50f'] - - id: image.repo_digests - type: string[] - brief: > - Repo digests of the container image as provided by the container runtime. - note: > - [Docker](https://docs.docker.com/engine/api/v1.43/#tag/Image/operation/ImageInspect) and - [CRI](https://github.com/kubernetes/cri-api/blob/c75ef5b473bbe2d0a4fc92f82235efd665ea8e9f/pkg/apis/runtime/v1/api.proto#L1237-L1238) - report those under the `RepoDigests` field. - examples: - - 'example@sha256:afcc7f1ac1b49db317a7196c902e61c6c3c4607d63599ee1a82d702d249a0ccb' - - 'internal.registry.example.com:5000/example@sha256:b69959407d21e8a062e0416bf13405bb2b71ed7a84dde4158ebafacfa06f5578' - - id: command - type: string - note: > - If using embedded credentials or sensitive data, it is recommended to remove them to prevent potential leakage. - brief: > - The command used to run the container (i.e. the command name). - examples: [ 'otelcontribcol' ] - - id: command_line - type: string - brief: > - The full command run by the container as a single string representing the full command. [2] - examples: [ 'otelcontribcol --config config.yaml' ] - - id: command_args - type: string[] - brief: > - All the command arguments (including the command/executable itself) run by the container. [2] - examples: [ 'otelcontribcol, --config, config.yaml' ] - - id: labels - type: template[string] - brief: > - Container labels, `` being the label name, the value being the label value. - examples: [ 'container.labels.app=nginx' ] diff --git a/internal/otelschema/_testdata/model/registry/db.yaml b/internal/otelschema/_testdata/model/registry/db.yaml deleted file mode 100644 index 3ec2d84c..00000000 --- a/internal/otelschema/_testdata/model/registry/db.yaml +++ /dev/null @@ -1,424 +0,0 @@ -groups: - - id: registry.db - prefix: db - type: attribute_group - brief: > - This document defines the attributes used to describe telemetry in the context of databases. - attributes: - - id: cassandra.coordinator.dc - type: string - brief: > - The data center of the coordinating node for a query. - examples: 'us-west-2' - tag: tech-specific-cassandra - - id: cassandra.coordinator.id - type: string - brief: > - The ID of the coordinating node for a query. - examples: 'be13faa2-8574-4d71-926d-27f16cf8a7af' - tag: tech-specific-cassandra - - id: cassandra.consistency_level - brief: > - The consistency level of the query. Based on consistency values from [CQL](https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/dml/dmlConfigConsistency.html). - type: - members: - - id: all - value: 'all' - - id: each_quorum - value: 'each_quorum' - - id: quorum - value: 'quorum' - - id: local_quorum - value: 'local_quorum' - - id: one - value: 'one' - - id: two - value: 'two' - - id: three - value: 'three' - - id: local_one - value: 'local_one' - - id: any - value: 'any' - - id: serial - value: 'serial' - - id: local_serial - value: 'local_serial' - tag: tech-specific-cassandra - - id: cassandra.idempotence - type: boolean - brief: > - Whether or not the query is idempotent. - tag: tech-specific-cassandra - - id: cassandra.page_size - type: int - brief: > - The fetch size used for paging, i.e. how many rows will be returned at once. - examples: [5000] - tag: tech-specific-cassandra - - id: cassandra.speculative_execution_count - type: int - brief: > - The number of times a query was speculatively executed. Not set or `0` if the query was not executed speculatively. - examples: [0, 2] - tag: tech-specific-cassandra - - id: cassandra.table - type: string - brief: The name of the primary Cassandra table that the operation is acting upon, including the keyspace name (if applicable). - note: > - This mirrors the db.sql.table attribute but references cassandra rather than sql. - It is not recommended to attempt any client-side parsing of - `db.statement` just to get this property, but it should be set if - it is provided by the library being instrumented. - If the operation is acting upon an anonymous table, or more than one table, this - value MUST NOT be set. - examples: 'mytable' - tag: tech-specific-cassandra - - id: connection_string - type: string - brief: > - The connection string used to connect to the database. - It is recommended to remove embedded credentials. - examples: 'Server=(localdb)\v11.0;Integrated Security=true;' - tag: db-generic - - id: cosmosdb.client_id - type: string - brief: Unique Cosmos client instance id. - examples: '3ba4827d-4422-483f-b59f-85b74211c11d' - tag: tech-specific-cosmosdb - - id: cosmosdb.connection_mode - type: - allow_custom_values: false - members: - - id: gateway - value: 'gateway' - brief: Gateway (HTTP) connections mode - - id: direct - value: 'direct' - brief: Direct connection. - brief: Cosmos client connection mode. - tag: tech-specific-cosmosdb - - id: cosmosdb.container - type: string - brief: Cosmos DB container name. - examples: 'anystring' - tag: tech-specific-cosmosdb - - id: cosmosdb.operation_type - type: - allow_custom_values: true - members: - - id: invalid - value: 'Invalid' - - id: create - value: 'Create' - - id: patch - value: 'Patch' - - id: read - value: 'Read' - - id: read_feed - value: 'ReadFeed' - - id: delete - value: 'Delete' - - id: replace - value: 'Replace' - - id: execute - value: 'Execute' - - id: query - value: 'Query' - - id: head - value: 'Head' - - id: head_feed - value: 'HeadFeed' - - id: upsert - value: 'Upsert' - - id: batch - value: 'Batch' - - id: query_plan - value: 'QueryPlan' - - id: execute_javascript - value: 'ExecuteJavaScript' - brief: CosmosDB Operation Type. - tag: tech-specific-cosmosdb - - id: cosmosdb.request_charge - type: double - brief: RU consumed for that operation - examples: [46.18, 1.0] - tag: tech-specific-cosmosdb - - id: cosmosdb.request_content_length - type: int - brief: Request payload size in bytes - tag: tech-specific-cosmosdb - - id: cosmosdb.status_code - type: int - brief: Cosmos DB status code. - examples: [200, 201] - tag: tech-specific-cosmosdb - - id: cosmosdb.sub_status_code - type: int - brief: Cosmos DB sub status code. - examples: [1000, 1002] - tag: tech-specific-cosmosdb - - id: elasticsearch.cluster.name - type: string - brief: > - Represents the identifier of an Elasticsearch cluster. - examples: ["e9106fc68e3044f0b1475b04bf4ffd5f"] - tag: tech-specific-elasticsearch - - id: elasticsearch.node.name - type: string - brief: > - Represents the human-readable identifier of the node/instance to which a request was routed. - examples: ["instance-0000000001"] - tag: tech-specific-elasticsearch - - id: elasticsearch.path_parts - type: template[string] - brief: > - A dynamic value in the url path. - note: > - Many Elasticsearch url paths allow dynamic values. These SHOULD be recorded in span attributes in the format - `db.elasticsearch.path_parts.`, where `` is the url path part name. The implementation SHOULD - reference the [elasticsearch schema](https://raw.githubusercontent.com/elastic/elasticsearch-specification/main/output/schema/schema.json) - in order to map the path part values to their names. - examples: ['db.elasticsearch.path_parts.index=test-index', 'db.elasticsearch.path_parts.doc_id=123'] - tag: tech-specific-elasticsearch - - id: jdbc.driver_classname - type: string - brief: > - The fully-qualified class name of the [Java Database Connectivity (JDBC)](https://docs.oracle.com/javase/8/docs/technotes/guides/jdbc/) driver used to connect. - examples: ['org.postgresql.Driver', 'com.microsoft.sqlserver.jdbc.SQLServerDriver'] - tag: tech-specific-jdbc - - id: mongodb.collection - type: string - brief: > - The MongoDB collection being accessed within the database stated in `db.name`. - examples: [ 'customers', 'products' ] - tag: tech-specific-mongodb - - id: mssql.instance_name - type: string - note: > - If setting a `db.mssql.instance_name`, `server.port` is no longer - required (but still recommended if non-standard). - brief: > - The Microsoft SQL Server [instance name](https://docs.microsoft.com/sql/connect/jdbc/building-the-connection-url?view=sql-server-ver15) - connecting to. This name is used to determine the port of a named instance. - examples: 'MSSQLSERVER' - tag: tech-specific-mssql - - id: name - type: string - brief: > - This attribute is used to report the name of the database being accessed. - For commands that switch the database, this should be set to the target database - (even if the command fails). - note: > - In some SQL databases, the database name to be used is called "schema name". - In case there are multiple layers that could be considered for database name - (e.g. Oracle instance name and schema name), - the database name to be used is the more specific layer (e.g. Oracle schema name). - examples: [ 'customers', 'main' ] - tag: db-generic - - id: operation - type: string - brief: > - The name of the operation being executed, e.g. the [MongoDB command name](https://docs.mongodb.com/manual/reference/command/#database-operations) - such as `findAndModify`, or the SQL keyword. - note: > - When setting this to an SQL keyword, it is not recommended to - attempt any client-side parsing of `db.statement` just to get this - property, but it should be set if the operation name is provided by - the library being instrumented. - If the SQL statement has an ambiguous operation, or performs more - than one operation, this value may be omitted. - examples: ['findAndModify', 'HMSET', 'SELECT'] - tag: db-generic - - id: redis.database_index - type: int - brief: > - The index of the database being accessed as used in the [`SELECT` command](https://redis.io/commands/select), provided as an integer. - To be used instead of the generic `db.name` attribute. - examples: [0, 1, 15] - tag: tech-specific-redis - - id: sql.table - type: string - brief: The name of the primary table that the operation is acting upon, including the database name (if applicable). - note: > - It is not recommended to attempt any client-side parsing of - `db.statement` just to get this property, but it should be set if - it is provided by the library being instrumented. - If the operation is acting upon an anonymous table, or more than one table, this - value MUST NOT be set. - examples: ['public.users', 'customers'] - tag: tech-specific-sql - - id: statement - type: string - brief: > - The database statement being executed. - examples: ['SELECT * FROM wuser_table', 'SET mykey "WuValue"'] - tag: db-generic - - id: system - brief: An identifier for the database management system (DBMS) product being used. See below for a list of well-known identifiers. - type: - allow_custom_values: true - members: - - id: other_sql - value: 'other_sql' - brief: 'Some other SQL database. Fallback only. See notes.' - - id: mssql - value: 'mssql' - brief: 'Microsoft SQL Server' - - id: mssqlcompact - value: 'mssqlcompact' - brief: 'Microsoft SQL Server Compact' - - id: mysql - value: 'mysql' - brief: 'MySQL' - - id: oracle - value: 'oracle' - brief: 'Oracle Database' - - id: db2 - value: 'db2' - brief: 'IBM Db2' - - id: postgresql - value: 'postgresql' - brief: 'PostgreSQL' - - id: redshift - value: 'redshift' - brief: 'Amazon Redshift' - - id: hive - value: 'hive' - brief: 'Apache Hive' - - id: cloudscape - value: 'cloudscape' - brief: 'Cloudscape' - - id: hsqldb - value: 'hsqldb' - brief: 'HyperSQL DataBase' - - id: progress - value: 'progress' - brief: 'Progress Database' - - id: maxdb - value: 'maxdb' - brief: 'SAP MaxDB' - - id: hanadb - value: 'hanadb' - brief: 'SAP HANA' - - id: ingres - value: 'ingres' - brief: 'Ingres' - - id: firstsql - value: 'firstsql' - brief: 'FirstSQL' - - id: edb - value: 'edb' - brief: 'EnterpriseDB' - - id: cache - value: 'cache' - brief: 'InterSystems Caché' - - id: adabas - value: 'adabas' - brief: 'Adabas (Adaptable Database System)' - - id: firebird - value: 'firebird' - brief: 'Firebird' - - id: derby - value: 'derby' - brief: 'Apache Derby' - - id: filemaker - value: 'filemaker' - brief: 'FileMaker' - - id: informix - value: 'informix' - brief: 'Informix' - - id: instantdb - value: 'instantdb' - brief: 'InstantDB' - - id: interbase - value: 'interbase' - brief: 'InterBase' - - id: mariadb - value: 'mariadb' - brief: 'MariaDB' - - id: netezza - value: 'netezza' - brief: 'Netezza' - - id: pervasive - value: 'pervasive' - brief: 'Pervasive PSQL' - - id: pointbase - value: 'pointbase' - brief: 'PointBase' - - id: sqlite - value: 'sqlite' - brief: 'SQLite' - - id: sybase - value: 'sybase' - brief: 'Sybase' - - id: teradata - value: 'teradata' - brief: 'Teradata' - - id: vertica - value: 'vertica' - brief: 'Vertica' - - id: h2 - value: 'h2' - brief: 'H2' - - id: coldfusion - value: 'coldfusion' - brief: 'ColdFusion IMQ' - - id: cassandra - value: 'cassandra' - brief: 'Apache Cassandra' - - id: hbase - value: 'hbase' - brief: 'Apache HBase' - - id: mongodb - value: 'mongodb' - brief: 'MongoDB' - - id: redis - value: 'redis' - brief: 'Redis' - - id: couchbase - value: 'couchbase' - brief: 'Couchbase' - - id: couchdb - value: 'couchdb' - brief: 'CouchDB' - - id: cosmosdb - value: 'cosmosdb' - brief: 'Microsoft Azure Cosmos DB' - - id: dynamodb - value: 'dynamodb' - brief: 'Amazon DynamoDB' - - id: neo4j - value: 'neo4j' - brief: 'Neo4j' - - id: geode - value: 'geode' - brief: 'Apache Geode' - - id: elasticsearch - value: 'elasticsearch' - brief: 'Elasticsearch' - - id: memcached - value: 'memcached' - brief: 'Memcached' - - id: cockroachdb - value: 'cockroachdb' - brief: 'CockroachDB' - - id: opensearch - value: 'opensearch' - brief: 'OpenSearch' - - id: clickhouse - value: 'clickhouse' - brief: 'ClickHouse' - - id: spanner - value: 'spanner' - brief: 'Cloud Spanner' - - id: trino - value: 'trino' - brief: 'Trino' - tag: db-generic - - id: user - type: string - brief: > - Username for accessing the database. - examples: ['readonly_user', 'reporting_user'] - tag: db-generic diff --git a/internal/otelschema/_testdata/model/registry/deprecated/http.yaml b/internal/otelschema/_testdata/model/registry/deprecated/http.yaml deleted file mode 100644 index 91503286..00000000 --- a/internal/otelschema/_testdata/model/registry/deprecated/http.yaml +++ /dev/null @@ -1,71 +0,0 @@ -groups: - - id: attributes.http.deprecated - type: attribute_group - brief: "Describes deprecated HTTP attributes." - prefix: http - attributes: - - id: method - type: string - brief: 'Deprecated, use `http.request.method` instead.' - stability: deprecated - examples: ["GET", "POST", "HEAD"] - - id: status_code - type: int - brief: 'Deprecated, use `http.response.status_code` instead.' - stability: deprecated - examples: [200] - - id: scheme - type: string - brief: 'Deprecated, use `url.scheme` instead.' - stability: deprecated - examples: ['http', 'https'] - - id: url - type: string - brief: 'Deprecated, use `url.full` instead.' - stability: deprecated - examples: ['https://www.foo.bar/search?q=OpenTelemetry#SemConv'] - - id: target - type: string - brief: 'Deprecated, use `url.path` and `url.query` instead.' - stability: deprecated - examples: ['/search?q=OpenTelemetry#SemConv'] - - id: request_content_length - type: int - brief: 'Deprecated, use `http.request.header.content-length` instead.' - stability: deprecated - examples: 3495 - - id: response_content_length - type: int - brief: 'Deprecated, use `http.response.header.content-length` instead.' - stability: deprecated - examples: 3495 - - id: flavor - type: - allow_custom_values: true - members: - - id: http_1_0 - value: '1.0' - brief: 'HTTP/1.0' - - id: http_1_1 - value: '1.1' - brief: 'HTTP/1.1' - - id: http_2_0 - value: '2.0' - brief: 'HTTP/2' - - id: http_3_0 - value: '3.0' - brief: 'HTTP/3' - - id: spdy - value: 'SPDY' - brief: 'SPDY protocol.' - - id: quic - value: 'QUIC' - brief: 'QUIC protocol.' - brief: 'Deprecated, use `network.protocol.name` instead.' - stability: deprecated - - id: user_agent - type: string - brief: 'Deprecated, use `user_agent.original` instead.' - examples: ['CERN-LineMode/2.15 libwww/2.17b3', - 'Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1'] - stability: deprecated diff --git a/internal/otelschema/_testdata/model/registry/deprecated/network.yaml b/internal/otelschema/_testdata/model/registry/deprecated/network.yaml deleted file mode 100644 index 19e9c226..00000000 --- a/internal/otelschema/_testdata/model/registry/deprecated/network.yaml +++ /dev/null @@ -1,100 +0,0 @@ -groups: - - id: network-deprecated - prefix: net - type: attribute_group - brief: > - These attributes may be used for any network related operation. - attributes: - - id: sock.peer.name - type: string - stability: deprecated - brief: Deprecated, no replacement at this time. - examples: ['/var/my.sock'] - - id: sock.peer.addr - type: string - stability: deprecated - brief: Deprecated, use `network.peer.address`. - examples: ['192.168.0.1'] - - id: sock.peer.port - type: int - stability: deprecated - examples: [65531] - brief: Deprecated, use `network.peer.port`. - - id: peer.name - type: string - stability: deprecated - brief: Deprecated, use `server.address` on client spans and `client.address` on server spans. - examples: ['example.com'] - - id: peer.port - type: int - stability: deprecated - brief: Deprecated, use `server.port` on client spans and `client.port` on server spans. - examples: [8080] - - id: host.name - type: string - stability: deprecated - brief: Deprecated, use `server.address`. - examples: ['example.com'] - - id: host.port - type: int - stability: deprecated - brief: Deprecated, use `server.port`. - examples: [8080] - - id: sock.host.addr - type: string - stability: deprecated - brief: Deprecated, use `network.local.address`. - examples: ['/var/my.sock'] - - id: sock.host.port - type: int - stability: deprecated - brief: Deprecated, use `network.local.port`. - examples: [8080] - - id: transport - type: - allow_custom_values: true - members: - - id: ip_tcp - value: "ip_tcp" - - id: ip_udp - value: "ip_udp" - - id: pipe - value: "pipe" - brief: 'Named or anonymous pipe.' - - id: inproc - value: "inproc" - brief: 'In-process communication.' - note: > - Signals that there is only in-process communication not using a "real" network protocol - in cases where network attributes would normally be expected. Usually all other network - attributes can be left out in that case. - - id: other - value: "other" - brief: 'Something else (non IP-based).' - stability: deprecated - brief: Deprecated, use `network.transport`. - - id: protocol.name - type: string - stability: deprecated - brief: Deprecated, use `network.protocol.name`. - examples: ['amqp', 'http', 'mqtt'] - - id: protocol.version - type: string - stability: deprecated - brief: Deprecated, use `network.protocol.version`. - examples: '3.1.1' - - id: sock.family - type: - allow_custom_values: true - members: - - id: inet - value: 'inet' - brief: "IPv4 address" - - id: inet6 - value: 'inet6' - brief: "IPv6 address" - - id: unix - value: 'unix' - brief: "Unix domain socket path" - stability: deprecated - brief: Deprecated, use `network.transport` and `network.type`. diff --git a/internal/otelschema/_testdata/model/registry/destination.yaml b/internal/otelschema/_testdata/model/registry/destination.yaml deleted file mode 100644 index 595a1c67..00000000 --- a/internal/otelschema/_testdata/model/registry/destination.yaml +++ /dev/null @@ -1,23 +0,0 @@ -groups: - - id: destination - prefix: destination - type: attribute_group - brief: > - These attributes may be used to describe the receiver of a network exchange/packet. These should be used - when there is no client/server relationship between the two sides, or when that relationship is unknown. - This covers low-level network interactions (e.g. packet tracing) where you don't know if - there was a connection or which side initiated it. - This also covers unidirectional UDP flows and peer-to-peer communication where the - "user-facing" surface of the protocol / API doesn't expose a clear notion of client and server. - attributes: - - id: address - type: string - brief: "Destination address - domain name if available without reverse DNS lookup; otherwise, IP address or Unix domain socket name." - note: > - When observed from the source side, and when communicating through an intermediary, `destination.address` SHOULD represent - the destination address behind any intermediaries, for example proxies, if it's available. - examples: ['destination.example.com', '10.1.2.80', '/tmp/my.sock'] - - id: port - type: int - brief: 'Destination port number' - examples: [3389, 2888] diff --git a/internal/otelschema/_testdata/model/registry/device.yaml b/internal/otelschema/_testdata/model/registry/device.yaml deleted file mode 100644 index 6eaf8e3b..00000000 --- a/internal/otelschema/_testdata/model/registry/device.yaml +++ /dev/null @@ -1,46 +0,0 @@ -groups: - - id: registry.device - prefix: device - type: resource - brief: > - Describes device attributes. - attributes: - - id: id - type: string - brief: > - A unique identifier representing the device - note: > - The device identifier MUST only be defined using the values outlined below. This value is not an advertising - identifier and MUST NOT be used as such. - On iOS (Swift or Objective-C), this value MUST be equal to the [vendor identifier](https://developer.apple.com/documentation/uikit/uidevice/1620059-identifierforvendor). - On Android (Java or Kotlin), this value MUST be equal to the Firebase Installation ID or a globally unique - UUID which is persisted across sessions in your application. More information can be found [here](https://developer.android.com/training/articles/user-data-ids) - on best practices and exact implementation details. - Caution should be taken when storing personal data or anything which can identify a user. GDPR and - data protection laws may apply, ensure you do your own due diligence. - examples: ['2ab2916d-a51f-4ac8-80ee-45ac31a28092'] - - id: manufacturer - type: string - brief: > - The name of the device manufacturer - note: > - The Android OS provides this field via [Build](https://developer.android.com/reference/android/os/Build#MANUFACTURER). - iOS apps SHOULD hardcode the value `Apple`. - examples: ['Apple', 'Samsung'] - - id: model.identifier - type: string - brief: > - The model identifier for the device - note: > - It's recommended this value represents a machine-readable version of - the model identifier rather than the market or consumer-friendly name - of the device. - examples: ['iPhone3,4', 'SM-G920F'] - - id: model.name - type: string - brief: > - The marketing name for the device model - note: > - It's recommended this value represents a human-readable version of the - device model rather than a machine-readable alternative. - examples: ['iPhone 6s Plus', 'Samsung Galaxy S6'] diff --git a/internal/otelschema/_testdata/model/registry/disk.yaml b/internal/otelschema/_testdata/model/registry/disk.yaml deleted file mode 100644 index 90d6fb27..00000000 --- a/internal/otelschema/_testdata/model/registry/disk.yaml +++ /dev/null @@ -1,17 +0,0 @@ -groups: - - id: registry.disk - prefix: disk - type: attribute_group - brief: > - These attributes may be used for any disk related operation. - attributes: - - id: io.direction - type: - allow_custom_values: false - members: - - id: read - value: 'read' - - id: write - value: 'write' - brief: "The disk IO operation direction." - examples: ["read"] diff --git a/internal/otelschema/_testdata/model/registry/error.yaml b/internal/otelschema/_testdata/model/registry/error.yaml deleted file mode 100644 index 683012e3..00000000 --- a/internal/otelschema/_testdata/model/registry/error.yaml +++ /dev/null @@ -1,35 +0,0 @@ -groups: - - id: registry.error - type: attribute_group - prefix: error - brief: > - This document defines the shared attributes used to report an error. - attributes: - - id: type - stability: stable - brief: > - Describes a class of error the operation ended with. - type: - allow_custom_values: true - members: - - id: other - value: "_OTHER" - brief: > - A fallback error value to be used when the instrumentation doesn't define a custom value. - examples: ['timeout', 'java.net.UnknownHostException', 'server_certificate_invalid', '500'] - note: | - The `error.type` SHOULD be predictable and SHOULD have low cardinality. - Instrumentations SHOULD document the list of errors they report. - - The cardinality of `error.type` within one instrumentation library SHOULD be low. - Telemetry consumers that aggregate data from multiple instrumentation libraries and applications - should be prepared for `error.type` to have high cardinality at query time when no - additional filters are applied. - - If the operation has completed successfully, instrumentations SHOULD NOT set `error.type`. - - If a specific domain defines its own set of error identifiers (such as HTTP or gRPC status codes), - it's RECOMMENDED to: - - * Use a domain-specific attribute - * Set `error.type` to capture all errors, regardless of whether they are defined within the domain-specific set or not. diff --git a/internal/otelschema/_testdata/model/registry/host.yaml b/internal/otelschema/_testdata/model/registry/host.yaml deleted file mode 100644 index 3deb5f57..00000000 --- a/internal/otelschema/_testdata/model/registry/host.yaml +++ /dev/null @@ -1,122 +0,0 @@ -groups: - - id: registry.host - prefix: host - type: resource - brief: > - A host is defined as a computing instance. For example, physical servers, virtual machines, switches or disk array. - attributes: - - id: id - type: string - brief: > - Unique host ID. For Cloud, this must be the instance_id assigned by the cloud provider. - For non-containerized systems, this should be the `machine-id`. See the table below for - the sources to use to determine the `machine-id` based on operating system. - examples: ['fdbf79e8af94cb7f9e8df36789187052'] - - id: name - type: string - brief: > - Name of the host. On Unix systems, it may contain what the hostname - command returns, or the fully qualified hostname, or another name - specified by the user. - examples: ['opentelemetry-test'] - - id: type - type: string - brief: > - Type of host. For Cloud, this must be the machine type. - examples: ['n1-standard-1'] - - id: arch - type: - allow_custom_values: true - members: - - id: amd64 - value: 'amd64' - brief: "AMD64" - - id: arm32 - value: 'arm32' - brief: "ARM32" - - id: arm64 - value: 'arm64' - brief: "ARM64" - - id: ia64 - value: 'ia64' - brief: "Itanium" - - id: ppc32 - value: 'ppc32' - brief: "32-bit PowerPC" - - id: ppc64 - value: 'ppc64' - brief: "64-bit PowerPC" - - id: s390x - value: 's390x' - brief: "IBM z/Architecture" - - id: x86 - value: 'x86' - brief: "32-bit x86" - brief: > - The CPU architecture the host system is running on. - - id: image.name - type: string - brief: > - Name of the VM image or OS install the host was instantiated from. - examples: ['infra-ami-eks-worker-node-7d4ec78312', 'CentOS-8-x86_64-1905'] - - id: image.id - type: string - brief: > - VM image ID or host OS image ID. For Cloud, this value is from the provider. - examples: ['ami-07b06b442921831e5'] - - id: image.version - type: string - brief: > - The version string of the VM image or host OS as defined in - [Version Attributes](/docs/resource/README.md#version-attributes). - examples: ['0.1'] - - id: ip - type: string[] - brief: > - Available IP addresses of the host, excluding loopback interfaces. - note: > - IPv4 Addresses MUST be specified in dotted-quad notation. IPv6 addresses - MUST be specified in the [RFC 5952](https://www.rfc-editor.org/rfc/rfc5952.html) format. - examples: ["192.168.1.140", "fe80::abc2:4a28:737a:609e"] - - id: mac - type: string[] - brief: > - Available MAC addresses of the host, excluding loopback interfaces. - note: > - MAC Addresses MUST be represented in [IEEE RA hexadecimal form](https://standards.ieee.org/wp-content/uploads/import/documents/tutorials/eui.pdf): - as hyphen-separated octets in uppercase hexadecimal form from most to least significant. - examples: ['AC-DE-48-23-45-67', 'AC-DE-48-23-45-67-01-9F'] - - id: cpu.vendor.id - type: string - brief: > - Processor manufacturer identifier. A maximum 12-character string. - note: > - [CPUID](https://wiki.osdev.org/CPUID) command returns the vendor ID string in EBX, EDX and ECX registers. - Writing these to memory in this order results in a 12-character string. - examples: [ 'GenuineIntel' ] - - id: cpu.family - type: string - brief: > - Family or generation of the CPU. - examples: [ '6', 'PA-RISC 1.1e' ] - - id: cpu.model.id - type: string - brief: > - Model identifier. It provides more granular information about the CPU, distinguishing it from - other CPUs within the same family. - examples: [ '6', '9000/778/B180L' ] - - id: cpu.model.name - type: string - brief: > - Model designation of the processor. - examples: [ '11th Gen Intel(R) Core(TM) i7-1185G7 @ 3.00GHz' ] - - id: cpu.stepping - type: int - brief: > - Stepping or core revisions. - examples: [ 1 ] - - id: cpu.cache.l2.size - type: int - brief: > - The amount of level 2 memory cache available to the processor (in Bytes). - examples: [ 12288000 ] diff --git a/internal/otelschema/_testdata/model/registry/http.yaml b/internal/otelschema/_testdata/model/registry/http.yaml deleted file mode 100644 index 2cb8e8b8..00000000 --- a/internal/otelschema/_testdata/model/registry/http.yaml +++ /dev/null @@ -1,135 +0,0 @@ -groups: - - id: registry.http - prefix: http - type: attribute_group - brief: 'This document defines semantic convention attributes in the HTTP namespace.' - attributes: - - id: request.body.size - type: int - brief: > - The size of the request payload body in bytes. This is the number of bytes transferred excluding headers and - is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) - header. For requests using transport encoding, this should be the compressed size. - examples: 3495 - stability: experimental # this should not be marked stable with other HTTP attributes - - id: request.header - stability: stable - type: template[string[]] - brief: > - HTTP request headers, `` being the normalized HTTP Header name (lowercase), the value being the header values. - note: > - Instrumentations SHOULD require an explicit configuration of which headers are to be captured. - Including all request headers can be a security risk - explicit configuration helps avoid leaking sensitive information. - - The `User-Agent` header is already captured in the `user_agent.original` attribute. - Users MAY explicitly configure instrumentations to capture them even though it is not recommended. - - The attribute value MUST consist of either multiple header values as an array of strings - or a single-item array containing a possibly comma-concatenated string, depending on the way - the HTTP library provides access to headers. - examples: ['http.request.header.content-type=["application/json"]', 'http.request.header.x-forwarded-for=["1.2.3.4", "1.2.3.5"]'] - - id: request.method - stability: stable - type: - allow_custom_values: true - members: - - id: connect - value: "CONNECT" - brief: 'CONNECT method.' - - id: delete - value: "DELETE" - brief: 'DELETE method.' - - id: get - value: "GET" - brief: 'GET method.' - - id: head - value: "HEAD" - brief: 'HEAD method.' - - id: options - value: "OPTIONS" - brief: 'OPTIONS method.' - - id: patch - value: "PATCH" - brief: 'PATCH method.' - - id: post - value: "POST" - brief: 'POST method.' - - id: put - value: "PUT" - brief: 'PUT method.' - - id: trace - value: "TRACE" - brief: 'TRACE method.' - - id: other - value: "_OTHER" - brief: 'Any HTTP method that the instrumentation has no prior knowledge of.' - brief: 'HTTP request method.' - examples: ["GET", "POST", "HEAD"] - note: | - HTTP request method value SHOULD be "known" to the instrumentation. - By default, this convention defines "known" methods as the ones listed in [RFC9110](https://www.rfc-editor.org/rfc/rfc9110.html#name-methods) - and the PATCH method defined in [RFC5789](https://www.rfc-editor.org/rfc/rfc5789.html). - - If the HTTP request method is not known to instrumentation, it MUST set the `http.request.method` attribute to `_OTHER`. - - If the HTTP instrumentation could end up converting valid HTTP request methods to `_OTHER`, then it MUST provide a way to override - the list of known HTTP methods. If this override is done via environment variable, then the environment variable MUST be named - OTEL_INSTRUMENTATION_HTTP_KNOWN_METHODS and support a comma-separated list of case-sensitive known HTTP methods - (this list MUST be a full override of the default known method, it is not a list of known methods in addition to the defaults). - - HTTP method names are case-sensitive and `http.request.method` attribute value MUST match a known HTTP method name exactly. - Instrumentations for specific web frameworks that consider HTTP methods to be case insensitive, SHOULD populate a canonical equivalent. - Tracing instrumentations that do so, MUST also set `http.request.method_original` to the original value. - - id: request.method_original - stability: stable - type: string - brief: Original HTTP method sent by the client in the request line. - examples: ["GeT", "ACL", "foo"] - - id: request.resend_count - stability: stable - type: int - brief: > - The ordinal number of request resending attempt (for any reason, including redirects). - note: > - The resend count SHOULD be updated each time an HTTP request gets resent by the client, regardless of what - was the cause of the resending (e.g. redirection, authorization failure, 503 Server Unavailable, network issues, - or any other). - examples: 3 - - id: response.body.size - type: int - brief: > - The size of the response payload body in bytes. This is the number of bytes transferred excluding headers and - is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) - header. For requests using transport encoding, this should be the compressed size. - examples: 3495 - stability: experimental # this should not be marked stable with other HTTP attributes - - id: response.header - stability: stable - type: template[string[]] - brief: > - HTTP response headers, `` being the normalized HTTP Header name (lowercase), the value being the header values. - note: > - Instrumentations SHOULD require an explicit configuration of which headers are to be captured. - Including all response headers can be a security risk - explicit configuration helps avoid leaking sensitive information. - - Users MAY explicitly configure instrumentations to capture them even though it is not recommended. - - The attribute value MUST consist of either multiple header values as an array of strings - or a single-item array containing a possibly comma-concatenated string, depending on the way - the HTTP library provides access to headers. - examples: ['http.response.header.content-type=["application/json"]', 'http.response.header.my-custom-header=["abc", "def"]'] - - id: response.status_code - stability: stable - type: int - brief: '[HTTP response status code](https://tools.ietf.org/html/rfc7231#section-6).' - examples: [200] - - id: route - stability: stable - type: string - brief: > - The matched route, that is, the path template in the format used by the respective server framework. - examples: ['/users/:userID?', '{controller}/{action}/{id?}'] - note: > - MUST NOT be populated when this is not supported by the HTTP server framework as the route attribute should have low-cardinality and the URI path can NOT substitute it. - - SHOULD include the [application root](/docs/http/http-spans.md#http-server-definitions) if there is one. diff --git a/internal/otelschema/_testdata/model/registry/k8s.yaml b/internal/otelschema/_testdata/model/registry/k8s.yaml deleted file mode 100644 index c42744cb..00000000 --- a/internal/otelschema/_testdata/model/registry/k8s.yaml +++ /dev/null @@ -1,140 +0,0 @@ -groups: - - id: registry.k8s - prefix: k8s - type: resource - brief: > - Kubernetes resource attributes. - attributes: - - id: cluster.name - type: string - brief: > - The name of the cluster. - examples: ['opentelemetry-cluster'] - - id: cluster.uid - type: string - brief: > - A pseudo-ID for the cluster, set to the UID of the `kube-system` - namespace. - note: | - K8s doesn't have support for obtaining a cluster ID. If this is ever - added, we will recommend collecting the `k8s.cluster.uid` through the - official APIs. In the meantime, we are able to use the `uid` of the - `kube-system` namespace as a proxy for cluster ID. Read on for the - rationale. - - Every object created in a K8s cluster is assigned a distinct UID. The - `kube-system` namespace is used by Kubernetes itself and will exist - for the lifetime of the cluster. Using the `uid` of the `kube-system` - namespace is a reasonable proxy for the K8s ClusterID as it will only - change if the cluster is rebuilt. Furthermore, Kubernetes UIDs are - UUIDs as standardized by - [ISO/IEC 9834-8 and ITU-T X.667](https://www.itu.int/ITU-T/studygroups/com17/oid.html). - Which states: - - > If generated according to one of the mechanisms defined in Rec. - ITU-T X.667 | ISO/IEC 9834-8, a UUID is either guaranteed to be - different from all other UUIDs generated before 3603 A.D., or is - extremely likely to be different (depending on the mechanism chosen). - - Therefore, UIDs between clusters should be extremely unlikely to - conflict. - examples: ['218fc5a9-a5f1-4b54-aa05-46717d0ab26d'] - - id: node.name - type: string - brief: > - The name of the Node. - examples: ['node-1'] - - id: node.uid - type: string - brief: > - The UID of the Node. - examples: ['1eb3a0c6-0477-4080-a9cb-0cb7db65c6a2'] - - id: namespace.name - type: string - brief: > - The name of the namespace that the pod is running in. - examples: ['default'] - - id: pod.uid - type: string - brief: > - The UID of the Pod. - examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] - - id: pod.name - type: string - brief: > - The name of the Pod. - examples: ['opentelemetry-pod-autoconf'] - - id: container.name - type: string - brief: > - The name of the Container from Pod specification, must be unique - within a Pod. Container runtime usually uses different globally unique - name (`container.name`). - examples: ['redis'] - - id: container.restart_count - type: int - brief: > - Number of times the container was restarted. This attribute can be - used to identify a particular container (running or stopped) within a - container spec. - examples: [0, 2] - - id: replicaset.uid - type: string - brief: > - The UID of the ReplicaSet. - examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] - - id: replicaset.name - type: string - brief: > - The name of the ReplicaSet. - examples: ['opentelemetry'] - - id: deployment.uid - type: string - brief: > - The UID of the Deployment. - examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] - - id: deployment.name - type: string - brief: > - The name of the Deployment. - examples: ['opentelemetry'] - - id: statefulset.uid - type: string - brief: > - The UID of the StatefulSet. - examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] - - id: statefulset.name - type: string - brief: > - The name of the StatefulSet. - examples: ['opentelemetry'] - - id: daemonset.uid - type: string - brief: > - The UID of the DaemonSet. - examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] - - id: daemonset.name - type: string - brief: > - The name of the DaemonSet. - examples: ['opentelemetry'] - - id: job.uid - type: string - brief: > - The UID of the Job. - examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] - - id: job.name - type: string - brief: > - The name of the Job. - examples: ['opentelemetry'] - - id: cronjob.uid - type: string - brief: > - The UID of the CronJob. - examples: ['275ecb36-5aa8-4c2a-9c47-d8bb681b9aff'] - - id: cronjob.name - type: string - brief: > - The name of the CronJob. - examples: ['opentelemetry'] diff --git a/internal/otelschema/_testdata/model/registry/messaging.yaml b/internal/otelschema/_testdata/model/registry/messaging.yaml deleted file mode 100644 index c7ba8fd4..00000000 --- a/internal/otelschema/_testdata/model/registry/messaging.yaml +++ /dev/null @@ -1,245 +0,0 @@ -groups: - - id: registry.messaging - prefix: messaging - type: attribute_group - brief: 'Attributes describing telemetry around messaging systems and messaging activities.' - attributes: - - id: batch.message_count - type: int - brief: The number of messages sent, received, or processed in the scope of the batching operation. - note: > - Instrumentations SHOULD NOT set `messaging.batch.message_count` on spans that operate with a single message. - When a messaging client library supports both batch and single-message API for the same operation, instrumentations SHOULD - use `messaging.batch.message_count` for batching APIs and SHOULD NOT use it for single-message APIs. - examples: [0, 1, 2] - - id: client_id - type: string - brief: > - A unique identifier for the client that consumes or produces a message. - examples: ['client-5', 'myhost@8742@s8083jm'] - - id: destination.name - type: string - brief: 'The message destination name' - note: | - Destination name SHOULD uniquely identify a specific queue, topic or other entity within the broker. If - the broker doesn't have such notion, the destination name SHOULD uniquely identify the broker. - examples: ['MyQueue', 'MyTopic'] - - id: destination.template - type: string - brief: Low cardinality representation of the messaging destination name - note: > - Destination names could be constructed from templates. - An example would be a destination name involving a user name or product id. - Although the destination name in this case is of high cardinality, - the underlying template is of low cardinality and can be effectively - used for grouping and aggregation. - examples: ['/customers/{customerId}'] - - id: destination.anonymous - type: boolean - brief: 'A boolean that is true if the message destination is anonymous (could be unnamed or have auto-generated name).' - - id: destination.temporary - type: boolean - brief: 'A boolean that is true if the message destination is temporary and might not exist anymore after messages are processed.' - - id: destination_publish.anonymous - type: boolean - brief: 'A boolean that is true if the publish message destination is anonymous (could be unnamed or have auto-generated name).' - - id: destination_publish.name - type: string - brief: 'The name of the original destination the message was published to' - note: | - The name SHOULD uniquely identify a specific queue, topic, or other entity within the broker. If - the broker doesn't have such notion, the original destination name SHOULD uniquely identify the broker. - examples: ['MyQueue', 'MyTopic'] - - id: kafka.consumer.group - type: string - brief: > - Name of the Kafka Consumer Group that is handling the message. - Only applies to consumers, not producers. - examples: 'my-group' - - id: kafka.destination.partition - type: int - brief: > - Partition the message is sent to. - examples: 2 - - id: kafka.message.key - type: string - brief: > - Message keys in Kafka are used for grouping alike messages to ensure they're processed on the same partition. - They differ from `messaging.message.id` in that they're not unique. - If the key is `null`, the attribute MUST NOT be set. - note: > - If the key type is not string, it's string representation has to be supplied for the attribute. - If the key has no unambiguous, canonical string form, don't include its value. - examples: 'myKey' - - id: kafka.message.offset - type: int - brief: > - The offset of a record in the corresponding Kafka partition. - examples: 42 - - id: kafka.message.tombstone - type: boolean - brief: 'A boolean that is true if the message is a tombstone.' - - id: message.conversation_id - type: string - brief: > - The conversation ID identifying the conversation to which the message belongs, - represented as a string. Sometimes called "Correlation ID". - examples: 'MyConversationId' - - id: message.envelope.size - type: int - brief: > - The size of the message body and metadata in bytes. - note: | - This can refer to both the compressed or uncompressed size. If both sizes are known, the uncompressed - size should be used. - examples: 2738 - - id: message.id - type: string - brief: 'A value used by the messaging system as an identifier for the message, represented as a string.' - examples: '452a7c7c7c7048c2f887f61572b18fc2' - - id: message.body.size - type: int - brief: > - The size of the message body in bytes. - note: | - This can refer to both the compressed or uncompressed body size. If both sizes are known, the uncompressed - body size should be used. - examples: 1439 - - id: operation - type: - allow_custom_values: true - members: - - id: publish - value: "publish" - brief: > - One or more messages are provided for publishing to an intermediary. - If a single message is published, the context of the "Publish" span can be used as the creation context and no "Create" span needs to be created. - - id: create - value: "create" - brief: > - A message is created. - "Create" spans always refer to a single message and are used to provide a unique creation context for messages in batch publishing scenarios. - - id: receive - value: "receive" - brief: > - One or more messages are requested by a consumer. - This operation refers to pull-based scenarios, where consumers explicitly call methods of messaging SDKs to receive messages. - - id: deliver - value: "deliver" - brief: > - One or more messages are passed to a consumer. - This operation refers to push-based scenarios, where consumer register callbacks which get called by messaging SDKs. - brief: > - A string identifying the kind of messaging operation. - note: If a custom value is used, it MUST be of low cardinality. - - id: rabbitmq.destination.routing_key - type: string - brief: > - RabbitMQ message routing key. - examples: 'myKey' - - id: rocketmq.client_group - type: string - brief: > - Name of the RocketMQ producer/consumer group that is handling the message. The client type is identified by the SpanKind. - examples: 'myConsumerGroup' - - id: rocketmq.consumption_model - type: - allow_custom_values: false - members: - - id: clustering - value: 'clustering' - brief: 'Clustering consumption model' - - id: broadcasting - value: 'broadcasting' - brief: 'Broadcasting consumption model' - brief: > - Model of message consumption. This only applies to consumer spans. - - id: rocketmq.message.delay_time_level - type: int - brief: > - The delay time level for delay message, which determines the message delay time. - examples: 3 - - id: rocketmq.message.delivery_timestamp - type: int - brief: > - The timestamp in milliseconds that the delay message is expected to be delivered to consumer. - examples: 1665987217045 - - id: rocketmq.message.group - type: string - brief: > - It is essential for FIFO message. Messages that belong to the same message group are always processed one by one within the same consumer group. - examples: 'myMessageGroup' - - id: rocketmq.message.keys - type: string[] - brief: > - Key(s) of message, another way to mark message besides message id. - examples: ['keyA', 'keyB'] - - id: rocketmq.message.tag - type: string - brief: > - The secondary classifier of message besides topic. - examples: tagA - - id: rocketmq.message.type - type: - allow_custom_values: false - members: - - id: normal - value: 'normal' - brief: "Normal message" - - id: fifo - value: 'fifo' - brief: 'FIFO message' - - id: delay - value: 'delay' - brief: 'Delay message' - - id: transaction - value: 'transaction' - brief: 'Transaction message' - brief: > - Type of message. - - id: rocketmq.namespace - type: string - brief: > - Namespace of RocketMQ resources, resources in different namespaces are individual. - examples: 'myNamespace' - - id: gcp_pubsub.message.ordering_key - type: string - brief: > - The ordering key for a given message. If the attribute is not present, the message does not have an ordering key. - examples: 'ordering_key' - - id: system - brief: > - An identifier for the messaging system being used. See below for a list of well-known identifiers. - type: - allow_custom_values: true - members: - - id: activemq - value: 'activemq' - brief: 'Apache ActiveMQ' - - id: aws_sqs - value: 'aws_sqs' - brief: 'Amazon Simple Queue Service (SQS)' - - id: azure_eventgrid - value: 'azure_eventgrid' - brief: 'Azure Event Grid' - - id: azure_eventhubs - value: 'azure_eventhubs' - brief: 'Azure Event Hubs' - - id: azure_servicebus - value: 'azure_servicebus' - brief: 'Azure Service Bus' - - id: gcp_pubsub - value: 'gcp_pubsub' - brief: 'Google Cloud Pub/Sub' - - id: jms - value: 'jms' - brief: 'Java Message Service' - - id: kafka - value: 'kafka' - brief: 'Apache Kafka' - - id: rabbitmq - value: 'rabbitmq' - brief: 'RabbitMQ' - - id: rocketmq - value: 'rocketmq' - brief: 'Apache RocketMQ' diff --git a/internal/otelschema/_testdata/model/registry/network.yaml b/internal/otelschema/_testdata/model/registry/network.yaml deleted file mode 100644 index 7715506b..00000000 --- a/internal/otelschema/_testdata/model/registry/network.yaml +++ /dev/null @@ -1,194 +0,0 @@ -groups: - - id: registry.network - prefix: network - type: attribute_group - brief: > - These attributes may be used for any network related operation. - attributes: - - id: carrier.icc - type: string - brief: "The ISO 3166-1 alpha-2 2-character country code associated with the mobile carrier network." - examples: "DE" - - id: carrier.mcc - type: string - brief: "The mobile carrier country code." - examples: "310" - - id: carrier.mnc - type: string - brief: "The mobile carrier network code." - examples: "001" - - id: carrier.name - type: string - brief: "The name of the mobile carrier." - examples: "sprint" - - id: connection.subtype - type: - allow_custom_values: true - members: - - id: gprs - brief: GPRS - value: "gprs" - - id: edge - brief: EDGE - value: "edge" - - id: umts - brief: UMTS - value: "umts" - - id: cdma - brief: CDMA - value: "cdma" - - id: evdo_0 - brief: EVDO Rel. 0 - value: "evdo_0" - - id: evdo_a - brief: "EVDO Rev. A" - value: "evdo_a" - - id: cdma2000_1xrtt - brief: CDMA2000 1XRTT - value: "cdma2000_1xrtt" - - id: hsdpa - brief: HSDPA - value: "hsdpa" - - id: hsupa - brief: HSUPA - value: "hsupa" - - id: hspa - brief: HSPA - value: "hspa" - - id: iden - brief: IDEN - value: "iden" - - id: evdo_b - brief: "EVDO Rev. B" - value: "evdo_b" - - id: lte - brief: LTE - value: "lte" - - id: ehrpd - brief: EHRPD - value: "ehrpd" - - id: hspap - brief: HSPAP - value: "hspap" - - id: gsm - brief: GSM - value: "gsm" - - id: td_scdma - brief: TD-SCDMA - value: "td_scdma" - - id: iwlan - brief: IWLAN - value: "iwlan" - - id: nr - brief: "5G NR (New Radio)" - value: "nr" - - id: nrnsa - brief: "5G NRNSA (New Radio Non-Standalone)" - value: "nrnsa" - - id: lte_ca - brief: LTE CA - value: "lte_ca" - brief: 'This describes more details regarding the connection.type. It may be the type of cell technology connection, but it could be used for describing details about a wifi connection.' - examples: 'LTE' - - id: connection.type - type: - allow_custom_values: true - members: - - id: wifi - value: "wifi" - - id: wired - value: "wired" - - id: cell - value: "cell" - - id: unavailable - value: "unavailable" - - id: unknown - value: "unknown" - brief: 'The internet connection type.' - examples: 'wifi' - - id: local.address - stability: stable - type: string - brief: Local address of the network connection - IP address or Unix domain socket name. - examples: ['10.1.2.80', '/tmp/my.sock'] - - id: local.port - stability: stable - type: int - brief: Local port number of the network connection. - examples: [65123] - - id: peer.address - stability: stable - type: string - brief: Peer address of the network connection - IP address or Unix domain socket name. - examples: ['10.1.2.80', '/tmp/my.sock'] - - id: peer.port - stability: stable - type: int - brief: Peer port number of the network connection. - examples: [65123] - - id: protocol.name - stability: stable - type: string - brief: '[OSI application layer](https://osi-model.com/application-layer/) or non-OSI equivalent.' - note: The value SHOULD be normalized to lowercase. - examples: ['amqp', 'http', 'mqtt'] - - id: protocol.version - stability: stable - type: string - brief: Version of the protocol specified in `network.protocol.name`. - examples: '3.1.1' - note: > - `network.protocol.version` refers to the version of the protocol used and might be - different from the protocol client's version. If the HTTP client has a version - of `0.27.2`, but sends HTTP version `1.1`, this attribute should be set to `1.1`. - - id: transport - stability: stable - type: - allow_custom_values: true - members: - - id: tcp - value: 'tcp' - brief: "TCP" - - id: udp - value: 'udp' - brief: "UDP" - - id: pipe - value: "pipe" - brief: 'Named or anonymous pipe.' - - id: unix - value: 'unix' - brief: "Unix domain socket" - brief: > - [OSI transport layer](https://osi-model.com/transport-layer/) or - [inter-process communication method](https://wikipedia.org/wiki/Inter-process_communication). - note: | - The value SHOULD be normalized to lowercase. - - Consider always setting the transport when setting a port number, since - a port number is ambiguous without knowing the transport. For example - different processes could be listening on TCP port 12345 and UDP port 12345. - examples: ['tcp', 'udp'] - - id: type - stability: stable - type: - allow_custom_values: true - members: - - id: ipv4 - value: 'ipv4' - brief: "IPv4" - - id: ipv6 - value: 'ipv6' - brief: "IPv6" - brief: '[OSI network layer](https://osi-model.com/network-layer/) or non-OSI equivalent.' - note: The value SHOULD be normalized to lowercase. - examples: ['ipv4', 'ipv6'] - - id: io.direction - type: - allow_custom_values: false - members: - - id: transmit - value: 'transmit' - - id: receive - value: 'receive' - brief: "The network IO operation direction." - examples: ["transmit"] diff --git a/internal/otelschema/_testdata/model/registry/oci.yaml b/internal/otelschema/_testdata/model/registry/oci.yaml deleted file mode 100644 index 45e28387..00000000 --- a/internal/otelschema/_testdata/model/registry/oci.yaml +++ /dev/null @@ -1,21 +0,0 @@ -groups: - - id: registry.oci.manifest - prefix: oci.manifest - type: resource - brief: > - An OCI image manifest. - attributes: - - id: digest - type: string - brief: > - The digest of the OCI image manifest. For container images specifically is the - digest by which the container image is known. - note: > - Follows - [OCI Image Manifest Specification](https://github.com/opencontainers/image-spec/blob/main/manifest.md), - and specifically the - [Digest property](https://github.com/opencontainers/image-spec/blob/main/descriptor.md#digests). - - An example can be found in - [Example Image Manifest](https://docs.docker.com/registry/spec/manifest-v2-2/#example-image-manifest). - examples: [ 'sha256:e4ca62c0d62f3e886e684806dfe9d4e0cda60d54986898173c1083856cfda0f4' ] diff --git a/internal/otelschema/_testdata/model/registry/os.yaml b/internal/otelschema/_testdata/model/registry/os.yaml deleted file mode 100644 index 5b699b1f..00000000 --- a/internal/otelschema/_testdata/model/registry/os.yaml +++ /dev/null @@ -1,69 +0,0 @@ -groups: - - id: registry.os - prefix: os - type: resource - brief: > - The operating system (OS) on which the process represented by this resource is running. - note: > - In case of virtualized environments, this is the operating system as it is observed by - the process, i.e., the virtualized guest rather than the underlying host. - attributes: - - id: type - type: - allow_custom_values: true - members: - - id: windows - value: 'windows' - brief: "Microsoft Windows" - - id: linux - value: 'linux' - brief: "Linux" - - id: darwin - value: 'darwin' - brief: "Apple Darwin" - - id: freebsd - value: 'freebsd' - brief: "FreeBSD" - - id: netbsd - value: 'netbsd' - brief: "NetBSD" - - id: openbsd - value: 'openbsd' - brief: "OpenBSD" - - id: dragonflybsd - value: 'dragonflybsd' - brief: "DragonFly BSD" - - id: hpux - value: 'hpux' - brief: "HP-UX (Hewlett Packard Unix)" - - id: aix - value: 'aix' - brief: "AIX (Advanced Interactive eXecutive)" - - id: solaris - value: 'solaris' - brief: "SunOS, Oracle Solaris" - - id: z_os - value: 'z_os' - brief: "IBM z/OS" - brief: > - The operating system type. - - id: description - type: string - brief: > - Human readable (not intended to be parsed) OS version information, - like e.g. reported by `ver` or `lsb_release -a` commands. - examples: ['Microsoft Windows [Version 10.0.18363.778]', 'Ubuntu 18.04.1 LTS'] - - id: name - type: string - brief: 'Human readable operating system name.' - examples: ['iOS', 'Android', 'Ubuntu'] - - id: version - type: string - brief: > - The version string of the operating system as defined in - [Version Attributes](/docs/resource/README.md#version-attributes). - examples: ['14.2.1', '18.04.1'] - - id: build_id - type: string - brief: 'Unique identifier for a particular build or compilation of the operating system.' - examples: ['TQ3C.230805.001.B2', '20E247', '22621'] diff --git a/internal/otelschema/_testdata/model/registry/process.yaml b/internal/otelschema/_testdata/model/registry/process.yaml deleted file mode 100644 index dd26c09b..00000000 --- a/internal/otelschema/_testdata/model/registry/process.yaml +++ /dev/null @@ -1,78 +0,0 @@ -groups: - - id: registry.process - prefix: process - type: resource - brief: > - An operating system process. - attributes: - - id: pid - type: int - brief: > - Process identifier (PID). - examples: [1234] - - id: parent_pid - type: int - brief: > - Parent Process identifier (PID). - examples: [111] - - id: executable.name - type: string - brief: > - The name of the process executable. On Linux based systems, can be set - to the `Name` in `proc/[pid]/status`. On Windows, can be set to the - base name of `GetProcessImageFileNameW`. - examples: ['otelcol'] - - id: executable.path - type: string - brief: > - The full path to the process executable. On Linux based systems, can - be set to the target of `proc/[pid]/exe`. On Windows, can be set to the - result of `GetProcessImageFileNameW`. - examples: ['/usr/bin/cmd/otelcol'] - - id: command - type: string - brief: > - The command used to launch the process (i.e. the command name). On Linux - based systems, can be set to the zeroth string in `proc/[pid]/cmdline`. - On Windows, can be set to the first parameter extracted from `GetCommandLineW`. - examples: ['cmd/otelcol'] - - id: command_line - type: string - brief: > - The full command used to launch the process as a single string representing - the full command. On Windows, can be set to the result of `GetCommandLineW`. - Do not set this if you have to assemble it just for monitoring; use - `process.command_args` instead. - examples: ['C:\cmd\otecol --config="my directory\config.yaml"'] - - id: command_args - type: string[] - brief: > - All the command arguments (including the command/executable itself) as - received by the process. On Linux-based systems (and some other Unixoid - systems supporting procfs), can be set according to the list of - null-delimited strings extracted from `proc/[pid]/cmdline`. For libc-based - executables, this would be the full argv vector passed to `main`. - examples: ['cmd/otecol', '--config=config.yaml'] - - id: owner - type: string - brief: > - The username of the user that owns the process. - examples: 'root' - - id: runtime.name - type: string - brief: > - The name of the runtime of this process. For compiled native binaries, - this SHOULD be the name of the compiler. - examples: ['OpenJDK Runtime Environment'] - - id: runtime.version - type: string - brief: > - The version of the runtime of this process, as returned by the runtime - without modification. - examples: '14.0.2' - - id: runtime.description - type: string - brief: > - An additional description about the runtime of the process, for example - a specific vendor customization of the runtime environment. - examples: 'Eclipse OpenJ9 Eclipse OpenJ9 VM openj9-0.21.0' diff --git a/internal/otelschema/_testdata/model/registry/rpc.yaml b/internal/otelschema/_testdata/model/registry/rpc.yaml deleted file mode 100644 index 64f1f9a6..00000000 --- a/internal/otelschema/_testdata/model/registry/rpc.yaml +++ /dev/null @@ -1,190 +0,0 @@ -groups: - - id: registry.rpc - prefix: rpc - type: attribute_group - brief: 'This document defines attributes for remote procedure calls.' - attributes: - - id: connect_rpc.error_code - type: - members: - - id: cancelled - value: cancelled - - id: unknown - value: unknown - - id: invalid_argument - value: invalid_argument - - id: deadline_exceeded - value: deadline_exceeded - - id: not_found - value: not_found - - id: already_exists - value: already_exists - - id: permission_denied - value: permission_denied - - id: resource_exhausted - value: resource_exhausted - - id: failed_precondition - value: failed_precondition - - id: aborted - value: aborted - - id: out_of_range - value: out_of_range - - id: unimplemented - value: unimplemented - - id: internal - value: internal - - id: unavailable - value: unavailable - - id: data_loss - value: data_loss - - id: unauthenticated - value: unauthenticated - brief: "The [error codes](https://connect.build/docs/protocol/#error-codes) of the Connect request. Error codes are always string values." - - id: connect_rpc.request.metadata - type: template[string[]] - brief: > - Connect request metadata, `` being the normalized Connect Metadata key (lowercase), the value being the metadata values. - note: > - Instrumentations SHOULD require an explicit configuration of which metadata values are to be captured. - Including all request metadata values can be a security risk - explicit configuration helps avoid leaking sensitive information. - examples: ['rpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"]'] - - id: connect_rpc.response.metadata - type: template[string[]] - brief: > - Connect response metadata, `` being the normalized Connect Metadata key (lowercase), the value being the metadata values. - note: > - Instrumentations SHOULD require an explicit configuration of which metadata values are to be captured. - Including all response metadata values can be a security risk - explicit configuration helps avoid leaking sensitive information. - examples: ['rpc.response.metadata.my-custom-metadata-attribute=["attribute_value"]'] - - id: grpc.status_code - type: - members: - - id: ok - brief: OK - value: 0 - - id: cancelled - brief: CANCELLED - value: 1 - - id: unknown - brief: UNKNOWN - value: 2 - - id: invalid_argument - brief: INVALID_ARGUMENT - value: 3 - - id: deadline_exceeded - brief: DEADLINE_EXCEEDED - value: 4 - - id: not_found - brief: NOT_FOUND - value: 5 - - id: already_exists - brief: ALREADY_EXISTS - value: 6 - - id: permission_denied - brief: PERMISSION_DENIED - value: 7 - - id: resource_exhausted - brief: RESOURCE_EXHAUSTED - value: 8 - - id: failed_precondition - brief: FAILED_PRECONDITION - value: 9 - - id: aborted - brief: ABORTED - value: 10 - - id: out_of_range - brief: OUT_OF_RANGE - value: 11 - - id: unimplemented - brief: UNIMPLEMENTED - value: 12 - - id: internal - brief: INTERNAL - value: 13 - - id: unavailable - brief: UNAVAILABLE - value: 14 - - id: data_loss - brief: DATA_LOSS - value: 15 - - id: unauthenticated - brief: UNAUTHENTICATED - value: 16 - brief: "The [numeric status code](https://github.com/grpc/grpc/blob/v1.33.2/doc/statuscodes.md) of the gRPC request." - - id: grpc.request.metadata - type: template[string[]] - brief: > - gRPC request metadata, `` being the normalized gRPC Metadata key (lowercase), the value being the metadata values. - note: > - Instrumentations SHOULD require an explicit configuration of which metadata values are to be captured. - Including all request metadata values can be a security risk - explicit configuration helps avoid leaking sensitive information. - examples: ['rpc.grpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"]'] - - id: grpc.response.metadata - type: template[string[]] - brief: > - gRPC response metadata, `` being the normalized gRPC Metadata key (lowercase), the value being the metadata values. - note: > - Instrumentations SHOULD require an explicit configuration of which metadata values are to be captured. - Including all response metadata values can be a security risk - explicit configuration helps avoid leaking sensitive information. - examples: ['rpc.grpc.response.metadata.my-custom-metadata-attribute=["attribute_value"]'] - - id: jsonrpc.error_code - type: int - brief: "`error.code` property of response if it is an error response." - examples: [-32700, 100] - - id: jsonrpc.error_message - type: string - brief: "`error.message` property of response if it is an error response." - examples: ['Parse error', 'User already exists'] - - id: jsonrpc.request_id - type: string - brief: > - `id` property of request or response. - Since protocol allows id to be int, string, `null` or missing (for notifications), - value is expected to be cast to string for simplicity. - Use empty string in case of `null` value. Omit entirely if this is a notification. - examples: ['10', 'request-7', ''] - - id: jsonrpc.version - type: string - brief: "Protocol version as in `jsonrpc` property of request/response. Since JSON-RPC 1.0 doesn't specify this, the value can be omitted." - examples: ['2.0', '1.0'] - - id: method - type: string - brief: 'The name of the (logical) method being called, must be equal to the $method part in the span name.' - note: > - This is the logical name of the method from the RPC interface perspective, - which can be different from the name of any implementing method/function. - The `code.function` attribute may be used to store the latter - (e.g., method actually executing the call on the server side, - RPC client stub method on the client side). - examples: "exampleMethod" - - id: service - type: string - brief: 'The full (logical) name of the service being called, including its package name, if applicable.' - note: > - This is the logical name of the service from the RPC interface perspective, - which can be different from the name of any implementing class. - The `code.namespace` attribute may be used to store the latter - (despite the attribute name, it may include a class name; - e.g., class with method actually executing the call on the server side, - RPC client stub class on the client side). - examples: "myservice.EchoService" - - id: system - brief: 'A string identifying the remoting system. See below for a list of well-known identifiers.' - type: - allow_custom_values: true - members: - - id: grpc - value: 'grpc' - brief: 'gRPC' - - id: java_rmi - value: 'java_rmi' - brief: 'Java RMI' - - id: dotnet_wcf - value: 'dotnet_wcf' - brief: '.NET WCF' - - id: apache_dubbo - value: 'apache_dubbo' - brief: 'Apache Dubbo' - - id: connect_rpc - value: 'connect_rpc' - brief: 'Connect RPC' diff --git a/internal/otelschema/_testdata/model/registry/server.yaml b/internal/otelschema/_testdata/model/registry/server.yaml deleted file mode 100644 index f47174dd..00000000 --- a/internal/otelschema/_testdata/model/registry/server.yaml +++ /dev/null @@ -1,28 +0,0 @@ -groups: - - id: server - prefix: server - type: attribute_group - brief: > - These attributes may be used to describe the server in a connection-based network interaction - where there is one side that initiates the connection (the client is the side that initiates the connection). - This covers all TCP network interactions since TCP is connection-based and one side initiates the - connection (an exception is made for peer-to-peer communication over TCP where the "user-facing" surface of the - protocol / API doesn't expose a clear notion of client and server). - This also covers UDP network interactions where one side initiates the interaction, e.g. QUIC (HTTP/3) and DNS. - attributes: - - id: address - stability: stable - type: string - brief: "Server domain name if available without reverse DNS lookup; otherwise, IP address or Unix domain socket name." - note: > - When observed from the client side, and when communicating through an intermediary, `server.address` SHOULD represent - the server address behind any intermediaries, for example proxies, if it's available. - examples: ['example.com', '10.1.2.80', '/tmp/my.sock'] - - id: port - stability: stable - type: int - brief: Server port number. - note: > - When observed from the client side, and when communicating through an intermediary, `server.port` SHOULD represent - the server port behind any intermediaries, for example proxies, if it's available. - examples: [80, 8080, 443] diff --git a/internal/otelschema/_testdata/model/registry/source.yaml b/internal/otelschema/_testdata/model/registry/source.yaml deleted file mode 100644 index 263a491c..00000000 --- a/internal/otelschema/_testdata/model/registry/source.yaml +++ /dev/null @@ -1,23 +0,0 @@ -groups: - - id: source - prefix: source - type: attribute_group - brief: > - These attributes may be used to describe the sender of a network exchange/packet. These should be used - when there is no client/server relationship between the two sides, or when that relationship is unknown. - This covers low-level network interactions (e.g. packet tracing) where you don't know if - there was a connection or which side initiated it. - This also covers unidirectional UDP flows and peer-to-peer communication where the - "user-facing" surface of the protocol / API doesn't expose a clear notion of client and server. - attributes: - - id: address - type: string - brief: "Source address - domain name if available without reverse DNS lookup; otherwise, IP address or Unix domain socket name." - note: > - When observed from the destination side, and when communicating through an intermediary, `source.address` SHOULD represent - the source address behind any intermediaries, for example proxies, if it's available. - examples: ['source.example.com', '10.1.2.80', '/tmp/my.sock'] - - id: port - type: int - brief: 'Source port number' - examples: [3389, 2888] diff --git a/internal/otelschema/_testdata/model/registry/thread.yaml b/internal/otelschema/_testdata/model/registry/thread.yaml deleted file mode 100644 index f6b5b5a5..00000000 --- a/internal/otelschema/_testdata/model/registry/thread.yaml +++ /dev/null @@ -1,17 +0,0 @@ -groups: - - id: registry.thread - prefix: thread - type: span - brief: > - These attributes may be used for any operation to store information about a thread that started a span. - attributes: - - id: id - type: int - brief: > - Current "managed" thread ID (as opposed to OS thread ID). - examples: 42 - - id: name - type: string - brief: > - Current thread name. - examples: main diff --git a/internal/otelschema/_testdata/model/registry/tls.yaml b/internal/otelschema/_testdata/model/registry/tls.yaml deleted file mode 100644 index 421479d1..00000000 --- a/internal/otelschema/_testdata/model/registry/tls.yaml +++ /dev/null @@ -1,165 +0,0 @@ -groups: - - id: registry.tls - prefix: tls - type: attribute_group - brief: "This document defines semantic convention attributes in the TLS namespace." - attributes: - - id: cipher - brief: > - String indicating the [cipher](https://datatracker.ietf.org/doc/html/rfc5246#appendix-A.5) used during the current connection. - type: string - note: > - The values allowed for `tls.cipher` MUST be one of the `Descriptions` of the - [registered TLS Cipher Suits](https://www.iana.org/assignments/tls-parameters/tls-parameters.xhtml#table-tls-parameters-4). - examples: - [ - "TLS_RSA_WITH_3DES_EDE_CBC_SHA", - "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256", - ] - - id: client.certificate - type: string - brief: > - PEM-encoded stand-alone certificate offered by the client. This is usually mutually-exclusive of `client.certificate_chain` since this value also exists in that list. - examples: ["MII..."] - - id: client.certificate_chain - type: string[] - brief: > - Array of PEM-encoded certificates that make up the certificate chain offered by the client. - This is usually mutually-exclusive of `client.certificate` since that value should be the first certificate in the chain. - examples: ["MII...", "MI..."] - - id: client.hash.md5 - type: string - brief: > - Certificate fingerprint using the MD5 digest of DER-encoded version of certificate offered by the client. - For consistency with other hash values, this value should be formatted as an uppercase hash. - examples: ["0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC"] - - id: client.hash.sha1 - type: string - brief: > - Certificate fingerprint using the SHA1 digest of DER-encoded version of certificate offered by the client. - For consistency with other hash values, this value should be formatted as an uppercase hash. - examples: ["9E393D93138888D288266C2D915214D1D1CCEB2A"] - - id: client.hash.sha256 - type: string - brief: > - Certificate fingerprint using the SHA256 digest of DER-encoded version of certificate offered by the client. - For consistency with other hash values, this value should be formatted as an uppercase hash. - examples: - ["0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0"] - - id: client.issuer - type: string - brief: "Distinguished name of [subject](https://datatracker.ietf.org/doc/html/rfc5280#section-4.1.2.6) of the issuer of the x.509 certificate presented by the client." - examples: - ["CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com"] - - id: client.ja3 - type: string - brief: "A hash that identifies clients based on how they perform an SSL/TLS handshake." - examples: ["d4e5b18d6b55c71272893221c96ba240"] - - id: client.not_after - type: string - brief: "Date/Time indicating when client certificate is no longer considered valid." - examples: ["2021-01-01T00:00:00.000Z"] - - id: client.not_before - type: string - brief: "Date/Time indicating when client certificate is first considered valid." - examples: ["1970-01-01T00:00:00.000Z"] - - id: client.server_name - type: string - brief: "Also called an SNI, this tells the server which hostname to which the client is attempting to connect to." - examples: ["opentelemetry.io"] - - id: client.subject - type: string - brief: "Distinguished name of subject of the x.509 certificate presented by the client." - examples: ["CN=myclient, OU=Documentation Team, DC=example, DC=com"] - - id: client.supported_ciphers - type: string[] - brief: Array of ciphers offered by the client during the client hello. - examples: - [ - '"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", "..."', - ] - - id: curve - brief: "String indicating the curve used for the given cipher, when applicable" - type: string - examples: ["secp256r1"] - - id: established - brief: "Boolean flag indicating if the TLS negotiation was successful and transitioned to an encrypted tunnel." - type: boolean - examples: [true] - - id: next_protocol - brief: > - String indicating the protocol being tunneled. - Per the values in the [IANA registry](https://www.iana.org/assignments/tls-extensiontype-values/tls-extensiontype-values.xhtml#alpn-protocol-ids), - this string should be lower case. - type: string - examples: ["http/1.1"] - - id: protocol.name - brief: > - Normalized lowercase protocol name parsed from original string of the negotiated [SSL/TLS protocol version](https://www.openssl.org/docs/man1.1.1/man3/SSL_get_version.html#RETURN-VALUES) - type: - allow_custom_values: true - members: - - id: ssl - value: ssl - - id: tls - value: tls - - id: protocol.version - brief: > - Numeric part of the version parsed from the original string of the negotiated [SSL/TLS protocol version](https://www.openssl.org/docs/man1.1.1/man3/SSL_get_version.html#RETURN-VALUES) - type: string - examples: ["1.2", "3"] - - id: resumed - brief: "Boolean flag indicating if this TLS connection was resumed from an existing TLS negotiation." - type: boolean - examples: [true] - - id: server.certificate - type: string - brief: > - PEM-encoded stand-alone certificate offered by the server. This is usually mutually-exclusive of `server.certificate_chain` since this value also exists in that list. - examples: ["MII..."] - - id: server.certificate_chain - type: string[] - brief: > - Array of PEM-encoded certificates that make up the certificate chain offered by the server. - This is usually mutually-exclusive of `server.certificate` since that value should be the first certificate in the chain. - examples: ["MII...", "MI..."] - - id: server.hash.md5 - type: string - brief: > - Certificate fingerprint using the MD5 digest of DER-encoded version of certificate offered by the server. - For consistency with other hash values, this value should be formatted as an uppercase hash. - examples: ["0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC"] - - id: server.hash.sha1 - type: string - brief: > - Certificate fingerprint using the SHA1 digest of DER-encoded version of certificate offered by the server. - For consistency with other hash values, this value should be formatted as an uppercase hash. - examples: ["9E393D93138888D288266C2D915214D1D1CCEB2A"] - - id: server.hash.sha256 - type: string - brief: > - Certificate fingerprint using the SHA256 digest of DER-encoded version of certificate offered by the server. - For consistency with other hash values, this value should be formatted as an uppercase hash. - examples: - ["0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0"] - - id: server.issuer - type: string - brief: "Distinguished name of [subject](https://datatracker.ietf.org/doc/html/rfc5280#section-4.1.2.6) of the issuer of the x.509 certificate presented by the client." - examples: - ["CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com"] - - id: server.ja3s - type: string - brief: "A hash that identifies servers based on how they perform an SSL/TLS handshake." - examples: ["d4e5b18d6b55c71272893221c96ba240"] - - id: server.not_after - type: string - brief: "Date/Time indicating when server certificate is no longer considered valid." - examples: ["2021-01-01T00:00:00.000Z"] - - id: server.not_before - type: string - brief: "Date/Time indicating when server certificate is first considered valid." - examples: ["1970-01-01T00:00:00.000Z"] - - id: server.subject - type: string - brief: "Distinguished name of subject of the x.509 certificate presented by the server." - examples: ["CN=myserver, OU=Documentation Team, DC=example, DC=com"] diff --git a/internal/otelschema/_testdata/model/registry/url.yaml b/internal/otelschema/_testdata/model/registry/url.yaml deleted file mode 100644 index 985ca912..00000000 --- a/internal/otelschema/_testdata/model/registry/url.yaml +++ /dev/null @@ -1,41 +0,0 @@ -groups: - - id: registry.url - brief: Attributes describing URL. - type: attribute_group - prefix: url - attributes: - - id: scheme - stability: stable - type: string - brief: 'The [URI scheme](https://www.rfc-editor.org/rfc/rfc3986#section-3.1) component identifying the used protocol.' - examples: ["https", "ftp", "telnet"] - - id: full - stability: stable - type: string - brief: Absolute URL describing a network resource according to [RFC3986](https://www.rfc-editor.org/rfc/rfc3986) - note: > - For network calls, URL usually has `scheme://host[:port][path][?query][#fragment]` format, where the fragment - is not transmitted over HTTP, but if it is known, it SHOULD be included nevertheless. - - `url.full` MUST NOT contain credentials passed via URL in form of `https://username:password@www.example.com/`. - In such case username and password SHOULD be redacted and attribute's value SHOULD be `https://REDACTED:REDACTED@www.example.com/`. - - `url.full` SHOULD capture the absolute URL when it is available (or can be reconstructed) - and SHOULD NOT be validated or modified except for sanitizing purposes. - examples: ['https://www.foo.bar/search?q=OpenTelemetry#SemConv', '//localhost'] - - id: path - stability: stable - type: string - brief: 'The [URI path](https://www.rfc-editor.org/rfc/rfc3986#section-3.3) component' - examples: ['/search'] - - id: query - stability: stable - type: string - brief: 'The [URI query](https://www.rfc-editor.org/rfc/rfc3986#section-3.4) component' - examples: ["q=OpenTelemetry"] - note: Sensitive content provided in query string SHOULD be scrubbed when instrumentations can identify it. - - id: fragment - stability: stable - type: string - brief: 'The [URI fragment](https://www.rfc-editor.org/rfc/rfc3986#section-3.5) component' - examples: ["SemConv"] diff --git a/internal/otelschema/_testdata/model/registry/user-agent.yaml b/internal/otelschema/_testdata/model/registry/user-agent.yaml deleted file mode 100644 index 0fbc5c9f..00000000 --- a/internal/otelschema/_testdata/model/registry/user-agent.yaml +++ /dev/null @@ -1,13 +0,0 @@ -groups: - - id: registry.user_agent - prefix: user_agent - type: attribute_group - brief: "Describes user-agent attributes." - attributes: - - id: original - stability: stable - type: string - brief: > - Value of the [HTTP User-Agent](https://www.rfc-editor.org/rfc/rfc9110.html#field.user-agent) header sent by the client. - examples: ['CERN-LineMode/2.15 libwww/2.17b3', - 'Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1'] diff --git a/internal/otelschema/_testdata/model/resource/android.yaml b/internal/otelschema/_testdata/model/resource/android.yaml deleted file mode 100644 index dcc236d8..00000000 --- a/internal/otelschema/_testdata/model/resource/android.yaml +++ /dev/null @@ -1,14 +0,0 @@ -groups: - - id: android - prefix: android - type: resource - brief: > - The Android platform on which the Android application is running. - attributes: - - id: os.api_level - type: string - brief: > - Uniquely identifies the framework API revision offered by a version - (`os.version`) of the android operating system. More information can be found - [here](https://developer.android.com/guide/topics/manifest/uses-sdk-element#ApiLevels). - examples: ['33', '32'] diff --git a/internal/otelschema/_testdata/model/resource/browser.yaml b/internal/otelschema/_testdata/model/resource/browser.yaml deleted file mode 100644 index 56830c1d..00000000 --- a/internal/otelschema/_testdata/model/resource/browser.yaml +++ /dev/null @@ -1,56 +0,0 @@ -groups: - - id: browser - prefix: browser - type: resource - brief: > - The web browser in which the application represented by the resource is running. - The `browser.*` attributes MUST be used only for resources that represent applications - running in a web browser (regardless of whether running on a mobile or desktop device). - attributes: - - id: brands - type: string[] - brief: 'Array of brand name and version separated by a space' - note: > - This value is intended to be taken from the - [UA client hints API](https://wicg.github.io/ua-client-hints/#interface) - (`navigator.userAgentData.brands`). - examples: [" Not A;Brand 99", "Chromium 99", "Chrome 99"] - - id: platform - type: string - brief: 'The platform on which the browser is running' - note: > - This value is intended to be taken from the - [UA client hints API](https://wicg.github.io/ua-client-hints/#interface) - (`navigator.userAgentData.platform`). If unavailable, the legacy - `navigator.platform` API SHOULD NOT be used instead and this attribute - SHOULD be left unset in order for the values to be consistent. - - The list of possible values is defined in the - [W3C User-Agent Client Hints specification](https://wicg.github.io/ua-client-hints/#sec-ch-ua-platform). - Note that some (but not all) of these values can overlap with values - in the [`os.type` and `os.name` attributes](./os.md). - However, for consistency, the values in the `browser.platform` attribute - should capture the exact value that the user agent provides. - examples: ['Windows', 'macOS', 'Android'] - - id: mobile - type: boolean - brief: 'A boolean that is true if the browser is running on a mobile device' - note: > - This value is intended to be taken from the - [UA client hints API](https://wicg.github.io/ua-client-hints/#interface) - (`navigator.userAgentData.mobile`). If unavailable, this attribute - SHOULD be left unset. - - id: language - type: string - brief: 'Preferred language of the user using the browser' - note: > - This value is intended to be taken from the Navigator API - `navigator.language`. - examples: ["en", "en-US", "fr", "fr-FR"] - - ref: user_agent.original - brief: 'Full user-agent string provided by the browser' - note: > - The user-agent value SHOULD be provided only from browsers that do not have a mechanism - to retrieve brands and platform individually from the User-Agent Client Hints API. - To retrieve the value, the legacy `navigator.userAgent` API can be used. - examples: ['Mozilla/5.0 (Macintosh; Intel Mac OS X 10_15_7) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/95.0.4638.54 Safari/537.36'] diff --git a/internal/otelschema/_testdata/model/resource/cloud.yaml b/internal/otelschema/_testdata/model/resource/cloud.yaml deleted file mode 100644 index 4699f848..00000000 --- a/internal/otelschema/_testdata/model/resource/cloud.yaml +++ /dev/null @@ -1,13 +0,0 @@ -groups: - - id: cloud - prefix: cloud - type: resource - brief: > - A cloud environment (e.g. GCP, Azure, AWS) - attributes: - - ref: cloud.provider - - ref: cloud.account.id - - ref: cloud.region - - ref: cloud.resource_id - - ref: cloud.availability_zone - - ref: cloud.platform diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/ecs.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/ecs.yaml deleted file mode 100644 index 2c6b8c9b..00000000 --- a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/ecs.yaml +++ /dev/null @@ -1,42 +0,0 @@ -groups: - - id: aws.ecs - prefix: aws.ecs - type: resource - brief: > - Resources used by AWS Elastic Container Service (ECS). - attributes: - - id: container.arn - type: string - brief: > - The Amazon Resource Name (ARN) of an [ECS container instance](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/ECS_instances.html). - examples: ['arn:aws:ecs:us-west-1:123456789123:container/32624152-9086-4f0e-acae-1a75b14fe4d9'] - - id: cluster.arn - type: string - brief: > - The ARN of an [ECS cluster](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/clusters.html). - examples: ['arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster'] - - id: launchtype - type: - allow_custom_values: false - members: - - id: ec2 - value: "ec2" - - id: fargate - value: "fargate" - brief: > - The [launch type](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/launch_types.html) for an ECS task. - - id: task.arn - type: string - brief: > - The ARN of an [ECS task definition](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task_definitions.html). - examples: ['arn:aws:ecs:us-west-1:123456789123:task/10838bed-421f-43ef-870a-f43feacbbb5b'] - - id: task.family - type: string - brief: > - The task definition family this task definition is a member of. - examples: ['opentelemetry-family'] - - id: task.revision - type: string - brief: > - The revision for this task definition. - examples: ["8", "26"] diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/eks.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/eks.yaml deleted file mode 100644 index 2c897253..00000000 --- a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/eks.yaml +++ /dev/null @@ -1,12 +0,0 @@ -groups: - - id: aws.eks - prefix: aws.eks - type: resource - brief: > - Resources used by AWS Elastic Kubernetes Service (EKS). - attributes: - - id: cluster.arn - type: string - brief: > - The ARN of an EKS cluster. - examples: ['arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster'] diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/logs.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/aws/logs.yaml deleted file mode 100644 index 8a433629..00000000 --- a/internal/otelschema/_testdata/model/resource/cloud_provider/aws/logs.yaml +++ /dev/null @@ -1,39 +0,0 @@ -groups: - - id: aws.log - prefix: aws.log - type: resource - brief: > - Resources specific to Amazon Web Services. - attributes: - - id: group.names - type: string[] - brief: > - The name(s) of the AWS log group(s) an application is writing to. - examples: ['/aws/lambda/my-function', 'opentelemetry-service'] - note: > - Multiple log groups must be supported for cases like multi-container applications, - where a single application has sidecar containers, and each write to their own log - group. - - id: group.arns - type: string[] - brief: > - The Amazon Resource Name(s) (ARN) of the AWS log group(s). - examples: ['arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:*'] - note: > - See the - [log group ARN format documentation](https://docs.aws.amazon.com/AmazonCloudWatch/latest/logs/iam-access-control-overview-cwl.html#CWL_ARN_Format). - - id: stream.names - type: string[] - brief: > - The name(s) of the AWS log stream(s) an application is writing to. - examples: ['logs/main/10838bed-421f-43ef-870a-f43feacbbb5b'] - - id: stream.arns - type: string[] - brief: > - The ARN(s) of the AWS log stream(s). - examples: ['arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:log-stream:logs/main/10838bed-421f-43ef-870a-f43feacbbb5b'] - note: > - See the - [log stream ARN format documentation](https://docs.aws.amazon.com/AmazonCloudWatch/latest/logs/iam-access-control-overview-cwl.html#CWL_ARN_Format). - One log group can contain several log streams, so these ARNs necessarily identify both a log - group and a log stream. diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/cloud_run.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/cloud_run.yaml deleted file mode 100644 index e4da8f59..00000000 --- a/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/cloud_run.yaml +++ /dev/null @@ -1,23 +0,0 @@ -groups: - - id: gcp.cloud_run - prefix: gcp.cloud_run - type: resource - brief: > - Resource used by Google Cloud Run. - attributes: - - id: job.execution - type: string - brief: > - The name of the Cloud Run - [execution](https://cloud.google.com/run/docs/managing/job-executions) - being run for the Job, as set by the - [`CLOUD_RUN_EXECUTION`](https://cloud.google.com/run/docs/container-contract#jobs-env-vars) - environment variable. - examples: ['job-name-xxxx', 'sample-job-mdw84'] - - id: job.task_index - type: int - brief: > - The index for a task within an execution as provided by the - [`CLOUD_RUN_TASK_INDEX`](https://cloud.google.com/run/docs/container-contract#jobs-env-vars) - environment variable. - examples: [0, 1] diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/gce.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/gce.yaml deleted file mode 100644 index 879d0ea3..00000000 --- a/internal/otelschema/_testdata/model/resource/cloud_provider/gcp/gce.yaml +++ /dev/null @@ -1,22 +0,0 @@ -groups: - - id: gcp.gce - prefix: gcp.gce - type: resource - brief: > - Resources used by Google Compute Engine (GCE). - attributes: - - id: instance.name - type: string - brief: > - The instance name of a GCE instance. This is the value - provided by `host.name`, the visible name of the instance in - the Cloud Console UI, and the prefix for the default - hostname of the instance as defined by the [default internal - DNS - name](https://cloud.google.com/compute/docs/internal-dns#instance-fully-qualified-domain-names). - examples: ['instance-1', 'my-vm-name'] - - id: instance.hostname - type: string - brief: > - The hostname of a GCE instance. This is the full value of the default or [custom hostname](https://cloud.google.com/compute/docs/instances/custom-hostname-vm). - examples: ['my-host1234.example.com', 'sample-vm.us-west1-b.c.my-project.internal'] diff --git a/internal/otelschema/_testdata/model/resource/cloud_provider/heroku.yaml b/internal/otelschema/_testdata/model/resource/cloud_provider/heroku.yaml deleted file mode 100644 index e73eddc1..00000000 --- a/internal/otelschema/_testdata/model/resource/cloud_provider/heroku.yaml +++ /dev/null @@ -1,25 +0,0 @@ -groups: - - id: heroku - prefix: heroku - type: resource - brief: > - Heroku dyno metadata - attributes: - - id: release.creation_timestamp - type: string - brief: > - Time and date the release was created - examples: [ '2022-10-23T18:00:42Z' ] - requirement_level: opt_in - - id: release.commit - type: string - brief: > - Commit hash for the current release - examples: [ 'e6134959463efd8966b20e75b913cafe3f5ec' ] - requirement_level: opt_in - - id: app.id - type: string - brief: > - Unique identifier for the application - examples: [ '2daa2797-e42b-4624-9322-ec3f968df4da' ] - requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/resource/container.yaml b/internal/otelschema/_testdata/model/resource/container.yaml deleted file mode 100644 index 97923e44..00000000 --- a/internal/otelschema/_testdata/model/resource/container.yaml +++ /dev/null @@ -1,22 +0,0 @@ -groups: - - id: container - prefix: container - type: resource - brief: > - A container instance. - attributes: - - ref: container.name - - ref: container.id - - ref: container.runtime - - ref: container.image.name - - ref: container.image.tags - - ref: container.image.id - - ref: container.image.repo_digests - - ref: container.command - requirement_level: opt_in - - ref: container.command_line - requirement_level: opt_in - - ref: container.command_args - requirement_level: opt_in - - ref: container.labels - - ref: oci.manifest.digest diff --git a/internal/otelschema/_testdata/model/resource/deployment_environment.yaml b/internal/otelschema/_testdata/model/resource/deployment_environment.yaml deleted file mode 100644 index 7ada3f97..00000000 --- a/internal/otelschema/_testdata/model/resource/deployment_environment.yaml +++ /dev/null @@ -1,21 +0,0 @@ -groups: - - id: deployment - prefix: deployment - type: resource - brief: > - The software deployment. - attributes: - - id: environment - type: string - brief: > - Name of the [deployment environment](https://wikipedia.org/wiki/Deployment_environment) - (aka deployment tier). - note: | - `deployment.environment` does not affect the uniqueness constraints defined through - the `service.namespace`, `service.name` and `service.instance.id` resource attributes. - This implies that resources carrying the following attribute combinations MUST be - considered to be identifying the same service: - - * `service.name=frontend`, `deployment.environment=production` - * `service.name=frontend`, `deployment.environment=staging`. - examples: ['staging', 'production'] diff --git a/internal/otelschema/_testdata/model/resource/device.yaml b/internal/otelschema/_testdata/model/resource/device.yaml deleted file mode 100644 index 8fab02a8..00000000 --- a/internal/otelschema/_testdata/model/resource/device.yaml +++ /dev/null @@ -1,11 +0,0 @@ -groups: - - id: device - prefix: device - type: resource - brief: > - The device on which the process represented by this resource is running. - attributes: - - ref: device.id - - ref: device.manufacturer - - ref: device.model.identifier - - ref: device.model.name diff --git a/internal/otelschema/_testdata/model/resource/faas.yaml b/internal/otelschema/_testdata/model/resource/faas.yaml deleted file mode 100644 index 7f1a02e0..00000000 --- a/internal/otelschema/_testdata/model/resource/faas.yaml +++ /dev/null @@ -1,63 +0,0 @@ -groups: - - id: faas_resource - prefix: faas - type: resource - brief: > - A serverless instance. - attributes: - - id: name - type: string - requirement_level: required - brief: > - The name of the single function that this runtime instance executes. - note: | - This is the name of the function as configured/deployed on the FaaS - platform and is usually different from the name of the callback - function (which may be stored in the - [`code.namespace`/`code.function`](/docs/general/attributes.md#source-code-attributes) - span attributes). - - For some cloud providers, the above definition is ambiguous. The following - definition of function name MUST be used for this attribute - (and consequently the span name) for the listed cloud providers/products: - - * **Azure:** The full name `/`, i.e., function app name - followed by a forward slash followed by the function name (this form - can also be seen in the resource JSON for the function). - This means that a span attribute MUST be used, as an Azure function - app can host multiple functions that would usually share - a TracerProvider (see also the `cloud.resource_id` attribute). - examples: ['my-function', 'myazurefunctionapp/some-function-name'] - - id: version - type: string - brief: The immutable version of the function being executed. - note: | - Depending on the cloud provider and platform, use: - - * **AWS Lambda:** The [function version](https://docs.aws.amazon.com/lambda/latest/dg/configuration-versions.html) - (an integer represented as a decimal string). - * **Google Cloud Run (Services):** The [revision](https://cloud.google.com/run/docs/managing/revisions) - (i.e., the function name plus the revision suffix). - * **Google Cloud Functions:** The value of the - [`K_REVISION` environment variable](https://cloud.google.com/functions/docs/env-var#runtime_environment_variables_set_automatically). - * **Azure Functions:** Not applicable. Do not set this attribute. - examples: ['26', 'pinkfroid-00002'] - - id: instance - type: string - brief: > - The execution environment ID as a string, that will be potentially reused - for other invocations to the same function/function version. - note: > - * **AWS Lambda:** Use the (full) log stream name. - examples: ['2021/06/28/[$LATEST]2f399eb14537447da05ab2a2e39309de'] - - id: max_memory - type: int - brief: > - The amount of memory available to the serverless function converted to Bytes. - note: > - It's recommended to set this attribute since e.g. too little memory can easily - stop a Java AWS Lambda function from working correctly. - On AWS Lambda, the environment variable `AWS_LAMBDA_FUNCTION_MEMORY_SIZE` - provides this information (which must be multiplied by 1,048,576). - examples: 134217728 - - ref: cloud.resource_id diff --git a/internal/otelschema/_testdata/model/resource/host.yaml b/internal/otelschema/_testdata/model/resource/host.yaml deleted file mode 100644 index b90b90dc..00000000 --- a/internal/otelschema/_testdata/model/resource/host.yaml +++ /dev/null @@ -1,37 +0,0 @@ -groups: - - id: host - prefix: host - type: resource - brief: > - A host is defined as a computing instance. For example, physical servers, virtual machines, switches or disk array. - attributes: - - ref: host.id - - ref: host.name - - ref: host.type - - ref: host.arch - - ref: host.image.name - - ref: host.image.id - - ref: host.image.version - - ref: host.ip - requirement_level: opt_in - - ref: host.mac - requirement_level: opt_in - - - id: host.cpu - prefix: host.cpu - type: resource - brief: > - A host's CPU information - attributes: - - ref: host.cpu.vendor.id - requirement_level: opt_in - - ref: host.cpu.family - requirement_level: opt_in - - ref: host.cpu.model.id - requirement_level: opt_in - - ref: host.cpu.model.name - requirement_level: opt_in - - ref: host.cpu.stepping - requirement_level: opt_in - - ref: host.cpu.cache.l2.size - requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/resource/k8s.yaml b/internal/otelschema/_testdata/model/resource/k8s.yaml deleted file mode 100644 index 20947c8a..00000000 --- a/internal/otelschema/_testdata/model/resource/k8s.yaml +++ /dev/null @@ -1,98 +0,0 @@ -groups: - - id: k8s.cluster - prefix: k8s.cluster - type: resource - brief: > - A Kubernetes Cluster. - attributes: - - ref: k8s.cluster.name - - ref: k8s.cluster.uid - - - id: k8s.node - prefix: k8s.node - type: resource - brief: > - A Kubernetes Node object. - attributes: - - ref: k8s.node.name - - ref: k8s.node.uid - - - id: k8s.namespace - prefix: k8s.namespace - type: resource - brief: > - A Kubernetes Namespace. - attributes: - - ref: k8s.namespace.name - - - id: k8s.pod - prefix: k8s.pod - type: resource - brief: > - A Kubernetes Pod object. - attributes: - - ref: k8s.pod.uid - - ref: k8s.pod.name - - - id: k8s.container - prefix: k8s.container - type: resource - brief: > - A container in a [PodTemplate](https://kubernetes.io/docs/concepts/workloads/pods/#pod-templates). - attributes: - - ref: k8s.container.name - - ref: k8s.container.restart_count - - - id: k8s.replicaset - prefix: k8s.replicaset - type: resource - brief: > - A Kubernetes ReplicaSet object. - attributes: - - ref: k8s.replicaset.uid - - ref: k8s.replicaset.name - - - id: k8s.deployment - prefix: k8s.deployment - type: resource - brief: > - A Kubernetes Deployment object. - attributes: - - ref: k8s.deployment.uid - - ref: k8s.deployment.name - - - id: k8s.statefulset - prefix: k8s.statefulset - type: resource - brief: > - A Kubernetes StatefulSet object. - attributes: - - ref: k8s.statefulset.uid - - ref: k8s.statefulset.name - - - id: k8s.daemonset - prefix: k8s.daemonset - type: resource - brief: > - A Kubernetes DaemonSet object. - attributes: - - ref: k8s.daemonset.uid - - ref: k8s.daemonset.name - - - id: k8s.job - prefix: k8s.job - type: resource - brief: > - A Kubernetes Job object. - attributes: - - ref: k8s.job.uid - - ref: k8s.job.name - - - id: k8s.cronjob - prefix: k8s.cronjob - type: resource - brief: > - A Kubernetes CronJob object. - attributes: - - ref: k8s.cronjob.uid - - ref: k8s.cronjob.name diff --git a/internal/otelschema/_testdata/model/resource/os.yaml b/internal/otelschema/_testdata/model/resource/os.yaml deleted file mode 100644 index 772fdde3..00000000 --- a/internal/otelschema/_testdata/model/resource/os.yaml +++ /dev/null @@ -1,16 +0,0 @@ -groups: - - id: os - prefix: os - type: resource - brief: > - The operating system (OS) on which the process represented by this resource is running. - note: > - In case of virtualized environments, this is the operating system as it is observed by - the process, i.e., the virtualized guest rather than the underlying host. - attributes: - - ref: os.type - requirement_level: required - - ref: os.description - - ref: os.name - - ref: os.version - - ref: os.build_id diff --git a/internal/otelschema/_testdata/model/resource/process.yaml b/internal/otelschema/_testdata/model/resource/process.yaml deleted file mode 100644 index 61223d61..00000000 --- a/internal/otelschema/_testdata/model/resource/process.yaml +++ /dev/null @@ -1,42 +0,0 @@ -groups: - - id: process - prefix: process - type: resource - brief: > - An operating system process. - attributes: - - ref: process.pid - - ref: process.parent_pid - - ref: process.executable.name - requirement_level: - conditionally_required: See alternative attributes below. - - ref: process.executable.path - requirement_level: - conditionally_required: See alternative attributes below. - - ref: process.command - requirement_level: - conditionally_required: See alternative attributes below. - - ref: process.command_line - requirement_level: - conditionally_required: See alternative attributes below. - - ref: process.command_args - requirement_level: - conditionally_required: See alternative attributes below. - - ref: process.owner - constraints: - - any_of: - - process.executable.name - - process.executable.path - - process.command - - process.command_line - - process.command_args - - - id: process.runtime - prefix: process.runtime - type: resource - brief: > - The single (language) runtime instance which is monitored. - attributes: - - ref: process.runtime.name - - ref: process.runtime.version - - ref: process.runtime.description diff --git a/internal/otelschema/_testdata/model/resource/service.yaml b/internal/otelschema/_testdata/model/resource/service.yaml deleted file mode 100644 index e930b621..00000000 --- a/internal/otelschema/_testdata/model/resource/service.yaml +++ /dev/null @@ -1,23 +0,0 @@ -groups: - - id: service - prefix: service - type: resource - brief: > - A service instance. - attributes: - - id: name - type: string - requirement_level: required - brief: > - Logical name of the service. - note: > - MUST be the same for all instances of horizontally scaled services. - If the value was not specified, SDKs MUST fallback to `unknown_service:` concatenated - with [`process.executable.name`](process.md#process), e.g. `unknown_service:bash`. - If `process.executable.name` is not available, the value MUST be set to `unknown_service`. - examples: ["shoppingcart"] - - id: version - type: string - brief: > - The version string of the service API or implementation. The format is not defined by these conventions. - examples: ["2.0.0", "a01dbef8a"] diff --git a/internal/otelschema/_testdata/model/resource/service_experimental.yaml b/internal/otelschema/_testdata/model/resource/service_experimental.yaml deleted file mode 100644 index 43c869ee..00000000 --- a/internal/otelschema/_testdata/model/resource/service_experimental.yaml +++ /dev/null @@ -1,37 +0,0 @@ -groups: - - id: service_experimental - prefix: service - type: resource - brief: > - A service instance. - attributes: - - id: namespace - type: string - brief: > - A namespace for `service.name`. - note: > - A string value having a meaning that helps to distinguish a group of services, - for example the team name that owns a group of services. - `service.name` is expected to be unique within the same namespace. - If `service.namespace` is not specified in the Resource then `service.name` - is expected to be unique for all services that have no explicit namespace defined - (so the empty/unspecified namespace is simply one more valid namespace). - Zero-length namespace string is assumed equal to unspecified namespace. - examples: ["Shop"] - - id: instance.id - type: string - brief: > - The string ID of the service instance. - note: > - MUST be unique for each instance of the same `service.namespace,service.name` pair - (in other words `service.namespace,service.name,service.instance.id` triplet MUST be globally unique). - The ID helps to distinguish instances of the same service that exist at the same time - (e.g. instances of a horizontally scaled service). It is preferable for the ID to be persistent - and stay the same for the lifetime of the service instance, however it is acceptable that - the ID is ephemeral and changes during important lifetime events for the service - (e.g. service restarts). - If the service has no inherent unique ID that can be used as the value of this attribute - it is recommended to generate a random Version 1 or Version 4 RFC 4122 UUID - (services aiming for reproducible UUIDs may also use Version 5, see RFC 4122 - for more recommendations). - examples: ["my-k8s-pod-deployment-1", "627cc493-f310-47de-96bd-71410b7dec09"] diff --git a/internal/otelschema/_testdata/model/resource/telemetry.yaml b/internal/otelschema/_testdata/model/resource/telemetry.yaml deleted file mode 100644 index 6966b4a8..00000000 --- a/internal/otelschema/_testdata/model/resource/telemetry.yaml +++ /dev/null @@ -1,57 +0,0 @@ -groups: - - id: telemetry - prefix: telemetry - type: resource - brief: > - The telemetry SDK used to capture data recorded by the instrumentation libraries. - attributes: - - id: sdk.name - type: string - requirement_level: required - brief: > - The name of the telemetry SDK as defined above. - note: | - The OpenTelemetry SDK MUST set the `telemetry.sdk.name` attribute to `opentelemetry`. - If another SDK, like a fork or a vendor-provided implementation, is used, this SDK MUST set the - `telemetry.sdk.name` attribute to the fully-qualified class or module name of this SDK's main entry point - or another suitable identifier depending on the language. - The identifier `opentelemetry` is reserved and MUST NOT be used in this case. - All custom identifiers SHOULD be stable across different versions of an implementation. - examples: ["opentelemetry"] - - id: sdk.language - type: - allow_custom_values: true - members: - - id: cpp - value: "cpp" - - id: dotnet - value: "dotnet" - - id: erlang - value: "erlang" - - id: go - value: "go" - - id: java - value: "java" - - id: nodejs - value: "nodejs" - - id: php - value: "php" - - id: python - value: "python" - - id: ruby - value: "ruby" - - id: rust - value: "rust" - - id: swift - value: "swift" - - id: webjs - value: "webjs" - requirement_level: required - brief: > - The language of the telemetry SDK. - - id: sdk.version - type: string - requirement_level: required - brief: > - The version string of the telemetry SDK. - examples: ["1.2.3"] diff --git a/internal/otelschema/_testdata/model/resource/telemetry_experimental.yaml b/internal/otelschema/_testdata/model/resource/telemetry_experimental.yaml deleted file mode 100644 index 8f7b2355..00000000 --- a/internal/otelschema/_testdata/model/resource/telemetry_experimental.yaml +++ /dev/null @@ -1,20 +0,0 @@ -groups: - - id: telemetry_experimental - prefix: telemetry - type: resource - brief: > - The telemetry SDK used to capture data recorded by the instrumentation libraries. - attributes: - - id: distro.name - type: string - brief: > - The name of the auto instrumentation agent or distribution, if used. - note: | - Official auto instrumentation agents and distributions SHOULD set the `telemetry.distro.name` attribute to - a string starting with `opentelemetry-`, e.g. `opentelemetry-java-instrumentation`. - examples: ["parts-unlimited-java"] - - id: distro.version - type: string - brief: > - The version string of the auto instrumentation agent or distribution, if used. - examples: ["1.2.3"] diff --git a/internal/otelschema/_testdata/model/resource/webengine.yaml b/internal/otelschema/_testdata/model/resource/webengine.yaml deleted file mode 100644 index 5e0cdf8f..00000000 --- a/internal/otelschema/_testdata/model/resource/webengine.yaml +++ /dev/null @@ -1,23 +0,0 @@ -groups: - - id: webengine_resource - prefix: webengine - type: resource - brief: > - Resource describing the packaged software running the application code. Web engines are typically executed using process.runtime. - attributes: - - id: name - type: string - requirement_level: required - brief: > - The name of the web engine. - examples: ['WildFly'] - - id: version - type: string - brief: > - The version of the web engine. - examples: ['21.0.0'] - - id: description - type: string - brief: > - Additional description of the web engine (e.g. detailed version and edition information). - examples: ['WildFly Full 21.0.0.Final (WildFly Core 13.0.1.Final) - 2.2.2.Final'] diff --git a/internal/otelschema/_testdata/model/scope/exporter/exporter.yaml b/internal/otelschema/_testdata/model/scope/exporter/exporter.yaml deleted file mode 100644 index 3bbf3fe0..00000000 --- a/internal/otelschema/_testdata/model/scope/exporter/exporter.yaml +++ /dev/null @@ -1,30 +0,0 @@ -groups: - - id: otel.scope - prefix: otel.scope - type: resource - brief: Attributes used by non-OTLP exporters to represent OpenTelemetry Scope's concepts. - attributes: - - id: name - type: string - brief: The name of the instrumentation scope - (`InstrumentationScope.Name` in OTLP). - examples: ['io.opentelemetry.contrib.mongodb'] - - id: version - type: string - brief: The version of the instrumentation scope - (`InstrumentationScope.Version` in OTLP). - examples: ['1.0.0'] - - id: otel.library - prefix: otel.library - type: resource - brief: > - Span attributes used by non-OTLP exporters to represent OpenTelemetry Scope's concepts. - attributes: - - id: name - type: string - stability: "deprecated" - brief: Deprecated, use the `otel.scope.name` attribute. - examples: ['io.opentelemetry.contrib.mongodb'] - - id: version - type: string - stability: "deprecated" - brief: Deprecated, use the `otel.scope.version` attribute. - examples: ['1.0.0'] diff --git a/internal/otelschema/_testdata/model/session.yaml b/internal/otelschema/_testdata/model/session.yaml deleted file mode 100644 index f2212653..00000000 --- a/internal/otelschema/_testdata/model/session.yaml +++ /dev/null @@ -1,26 +0,0 @@ -groups: - - id: session-id - prefix: session - type: attribute_group - brief: > - Session is defined as the period of time encompassing all activities performed by the application and the actions - executed by the end user. - - Consequently, a Session is represented as a collection of Logs, Events, and Spans emitted by the Client Application - throughout the Session's duration. Each Session is assigned a unique identifier, which is included as an attribute in - the Logs, Events, and Spans generated during the Session's lifecycle. - - When a session reaches end of life, typically due to user inactivity or session timeout, a new session identifier - will be assigned. The previous session identifier may be provided by the instrumentation so that telemetry - backends can link the two sessions. - attributes: - - id: id - type: string - brief: "A unique id to identify a session." - examples: "00112233-4455-6677-8899-aabbccddeeff" - requirement_level: opt_in - - id: previous_id - type: string - brief: "The previous `session.id` for this user, when known." - examples: "00112233-4455-6677-8899-aabbccddeeff" - requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/trace/aws/lambda.yaml b/internal/otelschema/_testdata/model/trace/aws/lambda.yaml deleted file mode 100644 index 73e77ea7..00000000 --- a/internal/otelschema/_testdata/model/trace/aws/lambda.yaml +++ /dev/null @@ -1,14 +0,0 @@ -groups: - - id: aws.lambda - prefix: aws.lambda - type: span - brief: > - Span attributes used by AWS Lambda (in addition to general `faas` attributes). - attributes: - - id: invoked_arn - type: string - brief: > - The full invoked ARN as provided on the `Context` passed to the function - (`Lambda-Runtime-Invoked-Function-Arn` header on the `/runtime/invocation/next` applicable). - note: This may be different from `cloud.resource_id` if an alias is involved. - examples: ['arn:aws:lambda:us-east-1:123456:function:myfunction:myalias'] diff --git a/internal/otelschema/_testdata/model/trace/cloudevents.yaml b/internal/otelschema/_testdata/model/trace/cloudevents.yaml deleted file mode 100644 index 6fcf7ae2..00000000 --- a/internal/otelschema/_testdata/model/trace/cloudevents.yaml +++ /dev/null @@ -1,36 +0,0 @@ -groups: - - id: cloudevents - prefix: cloudevents - type: span - brief: > - This document defines attributes for CloudEvents. - CloudEvents is a specification on how to define event data in a standard way. - These attributes can be attached to spans when performing operations with CloudEvents, regardless of the protocol being used. - attributes: - - id: event_id - type: string - requirement_level: required - brief: > - The [event_id](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#id) uniquely identifies the event. - examples: ['123e4567-e89b-12d3-a456-426614174000', '0001'] - - id: event_source - type: string - requirement_level: required - brief: > - The [source](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#source-1) identifies the context in which an event happened. - examples: ['https://github.com/cloudevents', '/cloudevents/spec/pull/123', 'my-service' ] - - id: event_spec_version - type: string - brief: > - The [version of the CloudEvents specification](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#specversion) which the event uses. - examples: '1.0' - - id: event_type - type: string - brief: > - The [event_type](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#type) contains a value describing the type of event related to the originating occurrence. - examples: ['com.github.pull_request.opened', 'com.example.object.deleted.v2'] - - id: event_subject - type: string - brief: > - The [subject](https://github.com/cloudevents/spec/blob/v1.0.2/cloudevents/spec.md#subject) of the event in the context of the event producer (identified by source). - examples: 'mynewfile.jpg' diff --git a/internal/otelschema/_testdata/model/trace/compatibility.yaml b/internal/otelschema/_testdata/model/trace/compatibility.yaml deleted file mode 100644 index 2e3fe12d..00000000 --- a/internal/otelschema/_testdata/model/trace/compatibility.yaml +++ /dev/null @@ -1,21 +0,0 @@ -groups: - - id: opentracing - prefix: opentracing - type: span - brief: 'This document defines semantic conventions for the OpenTracing Shim' - note: > - These conventions are used by the OpenTracing Shim layer. - attributes: - - id: ref_type - brief: 'Parent-child Reference type' - note: > - The causal relationship between a child Span and a parent Span. - type: - allow_custom_values: false - members: - - id: child_of - value: 'child_of' - brief: "The parent Span depends on the child Span in some capacity" - - id: follows_from - value: 'follows_from' - brief: "The parent Span doesn't depend in any way on the result of the child Span" diff --git a/internal/otelschema/_testdata/model/trace/database.yaml b/internal/otelschema/_testdata/model/trace/database.yaml deleted file mode 100644 index ed4f5716..00000000 --- a/internal/otelschema/_testdata/model/trace/database.yaml +++ /dev/null @@ -1,259 +0,0 @@ -groups: - - id: db - type: span - brief: > - This document defines the attributes used to perform database client calls. - span_kind: client - attributes: - - ref: db.system - tag: connection-level - requirement_level: required - - - ref: db.connection_string - tag: connection-level - - ref: db.user - tag: connection-level - - ref: db.jdbc.driver_classname - tag: connection-level-tech-specific - - ref: db.name - tag: call-level - requirement_level: - conditionally_required: If applicable. - - ref: db.statement - tag: call-level - requirement_level: - recommended: > - Should be collected by default only if there is sanitization that excludes sensitive information. - - ref: db.operation - tag: call-level - requirement_level: - conditionally_required: If `db.statement` is not applicable. - - ref: server.address - tag: connection-level - brief: > - Name of the database host. - - ref: server.port - tag: connection-level - requirement_level: - conditionally_required: If using a port other than the default port for this DBMS and if `server.address` is set. - - ref: network.peer.address - tag: connection-level - - ref: network.peer.port - requirement_level: - recommended: If `network.peer.address` is set. - tag: connection-level - - ref: network.transport - tag: connection-level - - ref: network.type - tag: connection-level - - - id: db.mssql - type: span - extends: db - brief: > - Connection-level attributes for Microsoft SQL Server - attributes: - - ref: db.mssql.instance_name - tag: connection-level-tech-specific - - - id: db.cassandra - type: span - extends: db - brief: > - Call-level attributes for Cassandra - attributes: - - ref: db.name - tag: call-level-tech-specific-cassandra - brief: > - The keyspace name in Cassandra. - examples: ["mykeyspace"] - note: For Cassandra the `db.name` should be set to the Cassandra keyspace name. - - ref: db.cassandra.page_size - tag: call-level-tech-specific-cassandra - - ref: db.cassandra.consistency_level - tag: call-level-tech-specific-cassandra - - ref: db.cassandra.table - tag: call-level-tech-specific-cassandra - - ref: db.cassandra.idempotence - tag: call-level-tech-specific-cassandra - - ref: db.cassandra.speculative_execution_count - tag: call-level-tech-specific-cassandra - - ref: db.cassandra.coordinator.id - tag: call-level-tech-specific-cassandra - - ref: db.cassandra.coordinator.dc - tag: call-level-tech-specific-cassandra - - - id: db.hbase - type: span - extends: db - brief: > - Call-level attributes for HBase - attributes: - - ref: db.name - tag: call-level-tech-specific - brief: > - The HBase namespace. - examples: ['mynamespace'] - note: For HBase the `db.name` should be set to the HBase namespace. - - - id: db.couchdb - type: span - extends: db - brief: > - Call-level attributes for CouchDB - attributes: - - ref: db.operation - tag: call-level-tech-specific - brief: > - The HTTP method + the target REST route. - examples: ['GET /{db}/{docid}'] - note: > - In **CouchDB**, `db.operation` should be set to the HTTP method + - the target REST route according to the API reference documentation. - For example, when retrieving a document, `db.operation` would be set to - (literally, i.e., without replacing the placeholders with concrete values): - [`GET /{db}/{docid}`](http://docs.couchdb.org/en/stable/api/document/common.html#get--db-docid). - - - id: db.redis - type: span - extends: db - brief: > - Call-level attributes for Redis - attributes: - - ref: db.redis.database_index - requirement_level: - conditionally_required: If other than the default database (`0`). - tag: call-level-tech-specific - - ref: db.statement - tag: call-level-tech-specific - brief: > - The full syntax of the Redis CLI command. - examples: ["HMSET myhash field1 'Hello' field2 'World'"] - note: > - For **Redis**, the value provided for `db.statement` SHOULD correspond to the syntax of the Redis CLI. - If, for example, the [`HMSET` command](https://redis.io/commands/hmset) is invoked, `"HMSET myhash field1 'Hello' field2 'World'"` would be a suitable value for `db.statement`. - - - id: db.mongodb - type: span - extends: db - brief: > - Call-level attributes for MongoDB - attributes: - - ref: db.mongodb.collection - requirement_level: required - tag: call-level-tech-specific - - - id: db.elasticsearch - type: span - extends: db - brief: > - Call-level attributes for Elasticsearch - attributes: - - ref: http.request.method - requirement_level: required - tag: call-level-tech-specific - - ref: db.operation - requirement_level: required - brief: The endpoint identifier for the request. - examples: [ 'search', 'ml.close_job', 'cat.aliases' ] - tag: call-level-tech-specific - - ref: url.full - requirement_level: required - examples: [ 'https://localhost:9200/index/_search?q=user.id:kimchy' ] - tag: call-level-tech-specific - - ref: db.statement - requirement_level: - recommended: > - Should be collected by default for search-type queries and only if there is sanitization that excludes - sensitive information. - brief: The request body for a [search-type query](https://www.elastic.co/guide/en/elasticsearch/reference/current/search.html), as a json string. - examples: [ '"{\"query\":{\"term\":{\"user.id\":\"kimchy\"}}}"' ] - tag: call-level-tech-specific - - ref: server.address - tag: call-level-tech-specific - - ref: server.port - tag: call-level-tech-specific - - ref: db.elasticsearch.cluster.name - requirement_level: - recommended: > - When communicating with an Elastic Cloud deployment, this should be collected from the "X-Found-Handling-Cluster" HTTP response header. - tag: call-level-tech-specific - - ref: db.elasticsearch.node.name - requirement_level: - recommended: > - When communicating with an Elastic Cloud deployment, this should be collected from the "X-Found-Handling-Instance" HTTP response header. - tag: call-level-tech-specific - - ref: db.elasticsearch.path_parts - requirement_level: - conditionally_required: when the url has dynamic values - tag: call-level-tech-specific - - - id: db.sql - type: span - extends: 'db' - brief: > - Call-level attributes for SQL databases - attributes: - - ref: db.sql.table - tag: call-level-tech-specific - - - id: db.cosmosdb - type: span - extends: db - prefix: db.cosmosdb - brief: > - Call-level attributes for Cosmos DB. - attributes: - - ref: db.cosmosdb.client_id - tag: call-level-tech-specific - - ref: db.cosmosdb.operation_type - requirement_level: - conditionally_required: when performing one of the operations in this list - tag: call-level-tech-specific - - ref: user_agent.original - brief: 'Full user-agent string is generated by Cosmos DB SDK' - note: > - The user-agent value is generated by SDK which is a combination of
- `sdk_version` : Current version of SDK. e.g. 'cosmos-netstandard-sdk/3.23.0'
- `direct_pkg_version` : Direct package version used by Cosmos DB SDK. e.g. '3.23.1'
- `number_of_client_instances` : Number of cosmos client instances created by the application. e.g. '1'
- `type_of_machine_architecture` : Machine architecture. e.g. 'X64'
- `operating_system` : Operating System. e.g. 'Linux 5.4.0-1098-azure 104 18'
- `runtime_framework` : Runtime Framework. e.g. '.NET Core 3.1.32'
- `failover_information` : Generated key to determine if region failover enabled. - Format Reg-{D (Disabled discovery)}-S(application region)|L(List of preferred regions)|N(None, user did not configure it). - Default value is "NS". - examples: ['cosmos-netstandard-sdk/3.23.0\|3.23.1\|1\|X64\|Linux 5.4.0-1098-azure 104 18\|.NET Core 3.1.32\|S\|'] - tag: call-level-tech-specific - - ref: db.cosmosdb.connection_mode - requirement_level: - conditionally_required: if not `direct` (or pick gw as default) - tag: call-level-tech-specific - - ref: db.cosmosdb.container - requirement_level: - conditionally_required: if available - tag: call-level-tech-specific - - ref: db.cosmosdb.request_content_length - tag: call-level-tech-specific - - ref: db.cosmosdb.status_code - requirement_level: - conditionally_required: if response was received - tag: call-level-tech-specific - - ref: db.cosmosdb.sub_status_code - requirement_level: - conditionally_required: when response was received and contained sub-code. - tag: call-level-tech-specific - - ref: db.cosmosdb.request_charge - requirement_level: - conditionally_required: when available - tag: call-level-tech-specific - - - id: db.tech - type: span - brief: "Semantic convention group for specific technologies" - constraints: - - include: 'db.cassandra' - - include: 'db.redis' - - include: 'db.mongodb' - - include: 'db.sql' - - include: 'db.cosmosdb' diff --git a/internal/otelschema/_testdata/model/trace/exporter/exporter.yaml b/internal/otelschema/_testdata/model/trace/exporter/exporter.yaml deleted file mode 100644 index 2989475d..00000000 --- a/internal/otelschema/_testdata/model/trace/exporter/exporter.yaml +++ /dev/null @@ -1,21 +0,0 @@ -groups: - - id: otel_span - prefix: otel - type: span - brief: Span attributes used by non-OTLP exporters to represent OpenTelemetry Span's concepts. - attributes: - - id: status_code - type: - allow_custom_values: false - members: - - id: ok - value: OK - brief: 'The operation has been validated by an Application developer or Operator to have completed successfully.' - - id: error - value: ERROR - brief: 'The operation contains an error.' - brief: Name of the code, either "OK" or "ERROR". MUST NOT be set if the status code is UNSET. - - id: status_description - type: string - brief: "Description of the Status if it has a value, otherwise not set." - examples: ['resource not found'] diff --git a/internal/otelschema/_testdata/model/trace/faas.yaml b/internal/otelschema/_testdata/model/trace/faas.yaml deleted file mode 100644 index 84b7141b..00000000 --- a/internal/otelschema/_testdata/model/trace/faas.yaml +++ /dev/null @@ -1,144 +0,0 @@ -groups: - - id: faas_span - prefix: faas - type: span - brief: > - This semantic convention describes an instance of a function that - runs without provisioning or managing of servers (also known as - serverless functions or Function as a Service (FaaS)) with spans. - attributes: - - ref: faas.trigger - note: | - For the server/consumer span on the incoming side, - `faas.trigger` MUST be set. - - Clients invoking FaaS instances usually cannot set `faas.trigger`, - since they would typically need to look in the payload to determine - the event type. If clients set it, it should be the same as the - trigger that corresponding incoming would have (i.e., this has - nothing to do with the underlying transport used to make the API - call to invoke the lambda, which is often HTTP). - - id: invocation_id - type: string - brief: 'The invocation ID of the current function invocation.' - examples: 'af9d5aa4-a685-4c5f-a22b-444f80b3cc28' - - ref: cloud.resource_id - - - id: faas_span.datasource - prefix: faas.document - type: span - brief: > - Semantic Convention for FaaS triggered as a response to some data - source operation such as a database or filesystem read/write. - attributes: - - id: collection - type: string - requirement_level: required - brief: > - The name of the source on which the triggering operation was performed. - For example, in Cloud Storage or S3 corresponds to the bucket name, - and in Cosmos DB to the database name. - examples: ['myBucketName', 'myDbName'] - - id: operation - requirement_level: required - type: - allow_custom_values: true - members: - - id: insert - value: 'insert' - brief: 'When a new object is created.' - - id: edit - value: 'edit' - brief: 'When an object is modified.' - - id: delete - value: 'delete' - brief: 'When an object is deleted.' - brief: 'Describes the type of the operation that was performed on the data.' - - id: time - type: string - brief: > - A string containing the time when the data was accessed in the - [ISO 8601](https://www.iso.org/iso-8601-date-and-time-format.html) - format expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). - examples: "2020-01-23T13:47:06Z" - - id: name - type: string - brief: > - The document name/table subjected to the operation. - For example, in Cloud Storage or S3 is the name of - the file, and in Cosmos DB the table name. - examples: ["myFile.txt", "myTableName"] - - - id: faas_span.http - type: span - brief: > - Semantic Convention for FaaS triggered as a response to some data - source operation such as a database or filesystem read/write. - constraints: - - include: trace.http.server - attributes: [] - - - id: faas_span.pubsub - type: span - brief: > - Semantic Convention for FaaS set to be executed when messages are - sent to a messaging system. - constraints: - - include: messaging - attributes: [] - - - id: faas_span.timer - prefix: faas - type: span - brief: > - Semantic Convention for FaaS scheduled to be executed regularly. - attributes: - - id: time - type: string - brief: > - A string containing the function invocation time in the - [ISO 8601](https://www.iso.org/iso-8601-date-and-time-format.html) - format expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). - examples: "2020-01-23T13:47:06Z" - - id: cron - type: string - brief: > - A string containing the schedule period as - [Cron Expression](https://docs.oracle.com/cd/E12058_01/doc/doc.1014/e12030/cron_expressions.htm). - examples: "0/5 * * * ? *" - - - id: faas_span.in - span_kind: server - prefix: faas - type: span - brief: > - Contains additional attributes for incoming FaaS spans. - attributes: - - id: coldstart - type: boolean - brief: > - A boolean that is true if the serverless function is executed for the - first time (aka cold-start). - - ref: faas.trigger - requirement_level: required - note: | - For the server/consumer span on the incoming side, - `faas.trigger` MUST be set. - - Clients invoking FaaS instances usually cannot set `faas.trigger`, - since they would typically need to look in the payload to determine - the event type. If clients set it, it should be the same as the - trigger that corresponding incoming would have (i.e., this has - nothing to do with the underlying transport used to make the API - call to invoke the lambda, which is often HTTP). - - - id: faas_span.out - span_kind: client - prefix: faas - type: span - brief: > - Contains additional attributes for outgoing FaaS spans. - attributes: - - ref: faas.invoked_name - - ref: faas.invoked_provider - - ref: faas.invoked_region diff --git a/internal/otelschema/_testdata/model/trace/feature-flag.yaml b/internal/otelschema/_testdata/model/trace/feature-flag.yaml deleted file mode 100644 index 50706ba2..00000000 --- a/internal/otelschema/_testdata/model/trace/feature-flag.yaml +++ /dev/null @@ -1,34 +0,0 @@ -groups: - - id: feature_flag - prefix: feature_flag - type: event - brief: > - This semantic convention defines the attributes used to represent a - feature flag evaluation as an event. - attributes: - - id: key - type: string - requirement_level: required - brief: The unique identifier of the feature flag. - examples: ["logo-color"] - - id: provider_name - type: string - requirement_level: recommended - brief: The name of the service provider that performs the flag evaluation. - examples: ["Flag Manager"] - - id: variant - type: string - requirement_level: recommended - examples: ["red", "true", "on"] - brief: > - SHOULD be a semantic identifier for a value. If one is unavailable, a - stringified version of the value can be used. - note: |- - A semantic identifier, commonly referred to as a variant, provides a means - for referring to a value without including the value itself. This can - provide additional context for understanding the meaning behind a value. - For example, the variant `red` maybe be used for the value `#c05543`. - - A stringified version of the value can be used in situations where a - semantic identifier is unavailable. String representation of the value - should be determined by the implementer. diff --git a/internal/otelschema/_testdata/model/trace/http.yaml b/internal/otelschema/_testdata/model/trace/http.yaml deleted file mode 100644 index 1e5667ad..00000000 --- a/internal/otelschema/_testdata/model/trace/http.yaml +++ /dev/null @@ -1,91 +0,0 @@ -groups: - - id: trace.http.common - extends: attributes.http.common - type: attribute_group - brief: 'This document defines semantic conventions for HTTP client and server Spans.' - note: > - These conventions can be used for http and https schemes - and various HTTP versions like 1.1, 2 and SPDY. - attributes: - - ref: http.request.method_original - requirement_level: - conditionally_required: If and only if it's different than `http.request.method`. - - ref: http.response.header - requirement_level: opt_in - - ref: http.request.method - sampling_relevant: true - requirement_level: required - - ref: network.peer.address - - ref: network.peer.port - requirement_level: - recommended: If `network.peer.address` is set. - - ref: network.transport - requirement_level: opt_in - note: > - Generally `tcp` for `HTTP/1.0`, `HTTP/1.1`, and `HTTP/2`. Generally `udp` for `HTTP/3`. - Other obscure implementations are possible. - - - id: trace.http.client - type: span - extends: attributes.http.client - span_kind: client - brief: 'Semantic Convention for HTTP Client' - attributes: - - ref: http.request.resend_count - requirement_level: - recommended: if and only if request was retried. - - ref: http.request.header - requirement_level: opt_in - - ref: server.address - sampling_relevant: true - - ref: server.port - sampling_relevant: true - - ref: url.full - sampling_relevant: true - requirement_level: required - - ref: user_agent.original - requirement_level: opt_in - - ref: url.scheme - - - id: trace.http.server - type: span - extends: attributes.http.server - span_kind: server - brief: 'Semantic Convention for HTTP Server' - attributes: - - ref: http.route - - ref: http.request.header - sampling_relevant: true - requirement_level: opt_in - - ref: server.address - sampling_relevant: true - - ref: server.port - sampling_relevant: true - - ref: network.local.address - requirement_level: opt_in - brief: Local socket address. Useful in case of a multi-IP host. - - ref: network.local.port - requirement_level: opt_in - brief: Local socket port. Useful in case of a multi-port host. - - ref: client.address - sampling_relevant: true - note: > - The IP address of the original client behind all proxies, if - known (e.g. from [Forwarded#for](https://developer.mozilla.org/docs/Web/HTTP/Headers/Forwarded#for), - [X-Forwarded-For](https://developer.mozilla.org/docs/Web/HTTP/Headers/X-Forwarded-For), or a similar header). - Otherwise, the immediate client peer address. - examples: ['83.164.160.102'] - - ref: client.port - requirement_level: opt_in - brief: The port of whichever client was captured in `client.address`. - - ref: url.path - requirement_level: required - sampling_relevant: true - - ref: url.query - requirement_level: - conditionally_required: If and only if one was received/sent. - sampling_relevant: true - - ref: url.scheme - sampling_relevant: true - - ref: user_agent.original - sampling_relevant: true diff --git a/internal/otelschema/_testdata/model/trace/instrumentation/aws-sdk.yml b/internal/otelschema/_testdata/model/trace/instrumentation/aws-sdk.yml deleted file mode 100644 index 90b6d1c9..00000000 --- a/internal/otelschema/_testdata/model/trace/instrumentation/aws-sdk.yml +++ /dev/null @@ -1,516 +0,0 @@ -groups: - - id: aws - prefix: aws - type: span - brief: > - The `aws` conventions apply to operations using the AWS SDK. They map request or response parameters - in AWS SDK API calls to attributes on a Span. The conventions have been collected over time based - on feedback from AWS users of tracing and will continue to evolve as new interesting conventions - are found. - - Some descriptions are also provided for populating general OpenTelemetry semantic conventions based on - these APIs. - attributes: - - ref: rpc.system - brief: "The value `aws-api`." - requirement_level: required - examples: - - aws-api - - ref: rpc.service - brief: "The name of the service to which a request is made, as returned by the AWS SDK." - examples: - - DynamoDB - - S3 - - ref: rpc.method - brief: "The name of the operation corresponding to the request, as returned by the AWS SDK" - examples: - - GetItem - - PutItem - - id: request_id - type: string - brief: "The AWS request ID as returned in the response headers `x-amz-request-id` or `x-amz-requestid`." - examples: - - 79b9da39-b7ae-508a-a6bc-864b2829c622 - - C9ER4AJX75574TDJ - - - id: dynamodb.all - type: span - brief: "Attributes always filled for all DynamoDB request types." - attributes: - - ref: db.system - brief: "The value `dynamodb`." - requirement_level: required - examples: - - dynamodb - - - id: dynamodb.shared - extends: aws - prefix: aws.dynamodb - type: span - brief: "Attributes that exist for multiple DynamoDB request types." - attributes: - - ref: db.operation - brief: "The same value as `rpc.method`." - examples: - - GetItem - - PutItem - - id: table_names - type: string[] - brief: The keys in the `RequestItems` object field. - examples: - - Users - - Cats - - id: consumed_capacity - type: string[] - brief: "The JSON-serialized value of each item in the `ConsumedCapacity` response field." - examples: - - '{ - "CapacityUnits": number, - "GlobalSecondaryIndexes": { - "string" : { - "CapacityUnits": number, - "ReadCapacityUnits": number, - "WriteCapacityUnits": number - } - }, - "LocalSecondaryIndexes": { - "string" : { - "CapacityUnits": number, - "ReadCapacityUnits": number, - "WriteCapacityUnits": number - } - }, - "ReadCapacityUnits": number, - "Table": { - "CapacityUnits": number, - "ReadCapacityUnits": number, - "WriteCapacityUnits": number - }, - "TableName": "string", - "WriteCapacityUnits": number - }' - - id: item_collection_metrics - type: string - brief: "The JSON-serialized value of the `ItemCollectionMetrics` response field." - examples: - - '{ - "string" : [ - { - "ItemCollectionKey": { - "string" : { - "B": blob, - "BOOL": boolean, - "BS": [ blob ], - "L": [ - "AttributeValue" - ], - "M": { - "string" : "AttributeValue" - }, - "N": "string", - "NS": [ "string" ], - "NULL": boolean, - "S": "string", - "SS": [ "string" ] - } - }, - "SizeEstimateRangeGB": [ number ] - } - ] - }' - - id: provisioned_read_capacity - type: double - brief: "The value of the `ProvisionedThroughput.ReadCapacityUnits` request parameter." - examples: - - 1.0 - - 2.0 - - id: provisioned_write_capacity - type: double - brief: "The value of the `ProvisionedThroughput.WriteCapacityUnits` request parameter." - examples: - - 1.0 - - 2.0 - - id: consistent_read - type: boolean - brief: "The value of the `ConsistentRead` request parameter." - - id: projection - type: string - brief: "The value of the `ProjectionExpression` request parameter." - examples: - - Title - - Title, Price, Color - - Title, Description, RelatedItems, ProductReviews - - id: limit - type: int - brief: "The value of the `Limit` request parameter." - examples: - - 10 - - id: attributes_to_get - type: string[] - brief: "The value of the `AttributesToGet` request parameter." - examples: - - lives - - id - - id: index_name - type: string - brief: "The value of the `IndexName` request parameter." - examples: - - name_to_group - - id: select - type: string - brief: "The value of the `Select` request parameter." - examples: - - ALL_ATTRIBUTES - - COUNT - - - id: dynamodb.batchgetitem - brief: DynamoDB.BatchGetItem - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - ref: aws.dynamodb.table_names - - ref: aws.dynamodb.consumed_capacity - - - id: dynamodb.batchwriteitem - brief: DynamoDB.BatchWriteItem - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - ref: aws.dynamodb.table_names - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.item_collection_metrics - - - id: dynamodb.createtable - brief: DynamoDB.CreateTable - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - id: global_secondary_indexes - type: string[] - brief: "The JSON-serialized value of each item of the `GlobalSecondaryIndexes` request field" - examples: - - '{ - "IndexName": "string", - "KeySchema": [ - { - "AttributeName": "string", - "KeyType": "string" - } - ], - "Projection": { - "NonKeyAttributes": [ "string" ], - "ProjectionType": "string" - }, - "ProvisionedThroughput": { - "ReadCapacityUnits": number, - "WriteCapacityUnits": number - } - }' - - id: local_secondary_indexes - type: string[] - brief: "The JSON-serialized value of each item of the `LocalSecondaryIndexes` request field." - examples: - - '{ - "IndexArn": "string", - "IndexName": "string", - "IndexSizeBytes": number, - "ItemCount": number, - "KeySchema": [ - { - "AttributeName": "string", - "KeyType": "string" - } - ], - "Projection": { - "NonKeyAttributes": [ "string" ], - "ProjectionType": "string" - } - }' - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.item_collection_metrics - - ref: aws.dynamodb.provisioned_read_capacity - - ref: aws.dynamodb.provisioned_write_capacity - - - id: dynamodb.deleteitem - brief: DynamoDB.DeleteItem - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.item_collection_metrics - - - id: dynamodb.deletetable - brief: DynamoDB.DeleteTable - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - - id: dynamodb.describetable - brief: DynamoDB.DescribeTable - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - - id: dynamodb.getitem - brief: DynamoDB.GetItem - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.consistent_read - - ref: aws.dynamodb.projection - - - id: dynamodb.listtables - brief: DynamoDB.ListTables - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - id: exclusive_start_table - type: string - brief: "The value of the `ExclusiveStartTableName` request parameter." - examples: - - Users - - CatsTable - - id: table_count - type: int - brief: "The the number of items in the `TableNames` response parameter." - examples: - - 20 - - ref: aws.dynamodb.limit - - - id: dynamodb.putitem - brief: DynamoDB.PutItem - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - ref: aws.dynamodb.table_names - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.item_collection_metrics - - - id: dynamodb.query - brief: DynamoDB.Query - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - id: scan_forward - type: boolean - brief: "The value of the `ScanIndexForward` request parameter." - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.consistent_read - - ref: aws.dynamodb.limit - - ref: aws.dynamodb.projection - - ref: aws.dynamodb.attributes_to_get - - ref: aws.dynamodb.index_name - - ref: aws.dynamodb.select - - - id: dynamodb.scan - brief: DynamoDB.Scan - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - id: segment - type: int - brief: "The value of the `Segment` request parameter." - examples: - - 10 - - id: total_segments - type: int - brief: "The value of the `TotalSegments` request parameter." - examples: - - 100 - - id: count - type: int - brief: "The value of the `Count` response parameter." - examples: - - 10 - - id: scanned_count - type: int - brief: "The value of the `ScannedCount` response parameter." - examples: - - 50 - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.consistent_read - - ref: aws.dynamodb.limit - - ref: aws.dynamodb.projection - - ref: aws.dynamodb.attributes_to_get - - ref: aws.dynamodb.index_name - - ref: aws.dynamodb.select - - - id: dynamodb.updateitem - brief: DynamoDB.UpdateItem - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.item_collection_metrics - - - id: dynamodb.updatetable - brief: DynamoDB.UpdateTable - extends: aws - prefix: aws.dynamodb - type: span - attributes: - - id: attribute_definitions - type: string[] - brief: "The JSON-serialized value of each item in the `AttributeDefinitions` request field." - examples: - - '{ - "AttributeName": "string", - "AttributeType": "string" - }' - - id: global_secondary_index_updates - type: string[] - brief: "The JSON-serialized value of each item in the the `GlobalSecondaryIndexUpdates` request field." - examples: - - '{ - "Create": { - "IndexName": "string", - "KeySchema": [ - { - "AttributeName": "string", - "KeyType": "string" - } - ], - "Projection": { - "NonKeyAttributes": [ "string" ], - "ProjectionType": "string" - }, - "ProvisionedThroughput": { - "ReadCapacityUnits": number, - "WriteCapacityUnits": number - } - }' - - ref: aws.dynamodb.table_names - brief: "A single-element array with the value of the TableName request parameter." - examples: - - Users - - ref: aws.dynamodb.consumed_capacity - - ref: aws.dynamodb.provisioned_read_capacity - - ref: aws.dynamodb.provisioned_write_capacity - - - id: aws.s3 - extends: aws - prefix: aws.s3 - type: span - brief: "Attributes that exist for S3 request types." - attributes: - - id: bucket - type: string - brief: "The S3 bucket name the request refers to. Corresponds to the `--bucket` parameter of the [S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/index.html) operations." - examples: - - some-bucket-name - note: | - The `bucket` attribute is applicable to all S3 operations that reference a bucket, i.e. that require the bucket name as a mandatory parameter. - This applies to almost all S3 operations except `list-buckets`. - - id: key - type: string - brief: "The S3 object key the request refers to. Corresponds to the `--key` parameter of the [S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/index.html) operations." - examples: - - someFile.yml - note: | - The `key` attribute is applicable to all object-related S3 operations, i.e. that require the object key as a mandatory parameter. - This applies in particular to the following operations: - - - [copy-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/copy-object.html) - - [delete-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/delete-object.html) - - [get-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/get-object.html) - - [head-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/head-object.html) - - [put-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/put-object.html) - - [restore-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/restore-object.html) - - [select-object-content](https://docs.aws.amazon.com/cli/latest/reference/s3api/select-object-content.html) - - [abort-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/abort-multipart-upload.html) - - [complete-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/complete-multipart-upload.html) - - [create-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/create-multipart-upload.html) - - [list-parts](https://docs.aws.amazon.com/cli/latest/reference/s3api/list-parts.html) - - [upload-part](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part.html) - - [upload-part-copy](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part-copy.html) - - id: copy_source - type: string - brief: "The source object (in the form `bucket`/`key`) for the copy operation." - examples: - - someFile.yml - note: | - The `copy_source` attribute applies to S3 copy operations and corresponds to the `--copy-source` parameter - of the [copy-object operation within the S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/copy-object.html). - This applies in particular to the following operations: - - - [copy-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/copy-object.html) - - [upload-part-copy](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part-copy.html) - - id: upload_id - type: string - brief: "Upload ID that identifies the multipart upload." - examples: - - 'dfRtDYWFbkRONycy.Yxwh66Yjlx.cph0gtNBtJ' - note: | - The `upload_id` attribute applies to S3 multipart-upload operations and corresponds to the `--upload-id` parameter - of the [S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/index.html) multipart operations. - This applies in particular to the following operations: - - - [abort-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/abort-multipart-upload.html) - - [complete-multipart-upload](https://docs.aws.amazon.com/cli/latest/reference/s3api/complete-multipart-upload.html) - - [list-parts](https://docs.aws.amazon.com/cli/latest/reference/s3api/list-parts.html) - - [upload-part](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part.html) - - [upload-part-copy](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part-copy.html) - - id: delete - type: string - brief: "The delete request container that specifies the objects to be deleted." - examples: - - 'Objects=[{Key=string,VersionId=string},{Key=string,VersionId=string}],Quiet=boolean' - note: | - The `delete` attribute is only applicable to the [delete-object](https://docs.aws.amazon.com/cli/latest/reference/s3api/delete-object.html) operation. - The `delete` attribute corresponds to the `--delete` parameter of the - [delete-objects operation within the S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/delete-objects.html). - - id: part_number - type: int - brief: "The part number of the part being uploaded in a multipart-upload operation. This is a positive integer between 1 and 10,000." - examples: - - 3456 - note: | - The `part_number` attribute is only applicable to the [upload-part](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part.html) - and [upload-part-copy](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part-copy.html) operations. - The `part_number` attribute corresponds to the `--part-number` parameter of the - [upload-part operation within the S3 API](https://docs.aws.amazon.com/cli/latest/reference/s3api/upload-part.html). diff --git a/internal/otelschema/_testdata/model/trace/instrumentation/graphql.yml b/internal/otelschema/_testdata/model/trace/instrumentation/graphql.yml deleted file mode 100644 index 88af65e9..00000000 --- a/internal/otelschema/_testdata/model/trace/instrumentation/graphql.yml +++ /dev/null @@ -1,32 +0,0 @@ -groups: - - id: graphql - prefix: graphql - type: span - brief: > - This document defines semantic conventions to apply when instrumenting the GraphQL implementation. They map - GraphQL operations to attributes on a Span. - attributes: - - id: operation.name - brief: "The name of the operation being executed." - type: string - examples: 'findBookById' - - id: operation.type - brief: "The type of the operation being executed." - type: - allow_custom_values: false - members: - - id: query - value: "query" - brief: "GraphQL query" - - id: mutation - value: "mutation" - brief: "GraphQL mutation" - - id: subscription - value: "subscription" - brief: "GraphQL subscription" - examples: ['query', 'mutation', 'subscription'] - - id: document - brief: "The GraphQL document being executed." - type: string - note: The value may be sanitized to exclude sensitive information. - examples: 'query findBookById { bookById(id: ?) { name } }' diff --git a/internal/otelschema/_testdata/model/trace/messaging.yaml b/internal/otelschema/_testdata/model/trace/messaging.yaml deleted file mode 100644 index 9163de0b..00000000 --- a/internal/otelschema/_testdata/model/trace/messaging.yaml +++ /dev/null @@ -1,182 +0,0 @@ -groups: - - id: messaging.message - type: attribute_group - brief: 'Semantic convention describing per-message attributes populated on messaging spans or links.' - attributes: - - ref: messaging.destination.name - - ref: messaging.message.id - - ref: messaging.message.conversation_id - - ref: messaging.message.envelope.size - - ref: messaging.message.body.size - - - id: messaging.destination - type: attribute_group - brief: 'Semantic convention for attributes that describe messaging destination on broker' - note: | - Destination attributes should be set on publish, receive, or other spans - describing messaging operations. - - Destination attributes should be set when the messaging operation handles - single messages. When the operation handles a batch of messages, - the destination attributes should only be applied when the attribute value - applies to all messages in the batch. - In other cases, destination attributes may be set on links. - attributes: - - ref: messaging.destination.name - - ref: messaging.destination.template - - ref: messaging.destination.temporary - - ref: messaging.destination.anonymous - - - id: messaging.destination_publish - prefix: messaging.destination_publish - type: attribute_group - brief: > - Semantic convention for attributes that describe the publish messaging destination on broker. - The term Publish Destination refers to the destination the message was originally published to. - These attributes should be used on the consumer side when information about - the publish destination is available and different than the destination message are consumed from. - note: | - Publish destination attributes should be set on publish, receive, - or other spans describing messaging operations. - Destination attributes should be set when the messaging operation handles - single messages. When the operation handles a batch of messages, - the destination attributes should only be applied when the attribute value - applies to all messages in the batch. - In other cases, destination attributes may be set on links. - attributes: - - ref: messaging.destination_publish.name - - ref: messaging.destination_publish.anonymous - - - id: messaging - type: span - brief: > - This document defines general attributes used in - messaging systems. - attributes: - - ref: messaging.system - requirement_level: required - - ref: messaging.operation - requirement_level: required - - ref: messaging.batch.message_count - requirement_level: - conditionally_required: If the span describes an operation on a batch of messages. - - ref: messaging.client_id - requirement_level: - recommended: If a client id is available - - ref: messaging.destination.name - requirement_level: - conditionally_required: If span describes operation on a single message or if the value applies to all messages in the batch. - - ref: messaging.destination.template - requirement_level: - conditionally_required: > - If available. Instrumentations MUST NOT use `messaging.destination.name` as template - unless low-cardinality of destination name is guaranteed. - - ref: messaging.destination.temporary - requirement_level: - conditionally_required: If value is `true`. When missing, the value is assumed to be `false`. - - ref: messaging.destination.anonymous - requirement_level: - conditionally_required: If value is `true`. When missing, the value is assumed to be `false`. - - ref: messaging.message.id - requirement_level: - recommended: Only for spans that represent an operation on a single message. - - ref: messaging.message.conversation_id - requirement_level: - recommended: Only if span represents operation on a single message. - - ref: messaging.message.envelope.size - requirement_level: - recommended: Only if span represents operation on a single message. - - ref: messaging.message.body.size - requirement_level: - recommended: Only if span represents operation on a single message. - - ref: server.address - note: > - This should be the IP/hostname of the broker (or other network-level peer) this specific message is sent to/received from. - requirement_level: - conditionally_required: If available. - - ref: network.peer.address - tag: connection-level - - ref: network.peer.port - requirement_level: - recommended: If `network.peer.address` is set. - tag: connection-level - - ref: network.transport - tag: connection-level - - ref: network.type - tag: connection-level - - ref: network.protocol.name - examples: ['amqp', 'mqtt'] - - ref: network.protocol.version - - - id: messaging.rabbitmq - type: attribute_group - extends: messaging - brief: > - Attributes for RabbitMQ - attributes: - - ref: messaging.rabbitmq.destination.routing_key - requirement_level: - conditionally_required: If not empty. - tag: tech-specific-rabbitmq - - - id: messaging.kafka - type: attribute_group - extends: messaging - brief: > - Attributes for Apache Kafka - attributes: - - ref: messaging.kafka.message.key - tag: tech-specific-kafka - - ref: messaging.kafka.consumer.group - tag: tech-specific-kafka - - ref: messaging.kafka.destination.partition - tag: tech-specific-kafka - - ref: messaging.kafka.message.offset - tag: tech-specific-kafka - - ref: messaging.kafka.message.tombstone - requirement_level: - conditionally_required: If value is `true`. When missing, the value is assumed to be `false`. - tag: tech-specific-kafka - - - id: messaging.rocketmq - type: attribute_group - extends: messaging - brief: > - Attributes for Apache RocketMQ - attributes: - - ref: messaging.rocketmq.namespace - requirement_level: required - tag: tech-specific-rocketmq - - ref: messaging.rocketmq.client_group - requirement_level: required - tag: tech-specific-rocketmq - - ref: messaging.rocketmq.message.delivery_timestamp - requirement_level: - conditionally_required: If the message type is delay and delay time level is not specified. - tag: tech-specific-rocketmq - - ref: messaging.rocketmq.message.delay_time_level - requirement_level: - conditionally_required: If the message type is delay and delivery timestamp is not specified. - tag: tech-specific-rocketmq - - ref: messaging.rocketmq.message.group - requirement_level: - conditionally_required: If the message type is FIFO. - tag: tech-specific-rocketmq - - ref: messaging.rocketmq.message.type - tag: tech-specific-rocketmq - - ref: messaging.rocketmq.message.tag - tag: tech-specific-rocketmq - - ref: messaging.rocketmq.message.keys - tag: tech-specific-rocketmq - - ref: messaging.rocketmq.consumption_model - tag: tech-specific-rocketmq - - id: messaging.gcp_pubsub - type: attribute_group - extends: messaging - brief: > - Attributes for Google Cloud Pub/Sub - attributes: - - ref: messaging.gcp_pubsub.message.ordering_key - tag: tech-specific-gcp-pubsub - requirement_level: - conditionally_required: If the message type has an ordering key set. diff --git a/internal/otelschema/_testdata/model/trace/rpc.yaml b/internal/otelschema/_testdata/model/trace/rpc.yaml deleted file mode 100644 index de36d676..00000000 --- a/internal/otelschema/_testdata/model/trace/rpc.yaml +++ /dev/null @@ -1,129 +0,0 @@ -groups: - - id: rpc - prefix: rpc - type: span - brief: 'This document defines semantic conventions for remote procedure calls.' - events: [rpc.message] - attributes: - - ref: rpc.system - requirement_level: required - - ref: rpc.service - - ref: rpc.method - - ref: network.transport - - ref: network.type - - ref: server.address - requirement_level: required - brief: > - RPC server [host name](https://grpc.github.io/grpc/core/md_doc_naming.html). - note: > - May contain server IP address, DNS name, or local socket name. When host component is an IP address, - instrumentations SHOULD NOT do a reverse proxy lookup to obtain DNS name and SHOULD set - `server.address` to the IP address provided in the host component. - - ref: server.port - requirement_level: - conditionally_required: if the port is supported by the network transport used for communication. - - - id: rpc.client - type: span - brief: 'This document defines semantic conventions for remote procedure call client spans.' - extends: rpc - attributes: - - ref: network.peer.address - - ref: network.peer.port - requirement_level: - recommended: If `network.peer.address` is set. - - - id: rpc.server - type: span - extends: rpc - span_kind: server - brief: 'Semantic Convention for RPC server spans' - attributes: - - ref: client.address - - ref: client.port - - ref: network.peer.address - - ref: network.peer.port - requirement_level: - recommended: If `network.peer.address` is set. - - ref: network.transport - - ref: network.type - - - id: rpc.grpc - type: span - extends: rpc - brief: 'Tech-specific attributes for gRPC.' - attributes: - - ref: rpc.grpc.status_code - tag: grpc-tech-specific - requirement_level: required - - ref: rpc.grpc.request.metadata - tag: grpc-tech-specific - requirement_level: opt_in - - ref: rpc.grpc.response.metadata - tag: grpc-tech-specific - requirement_level: opt_in - - - id: rpc.jsonrpc - prefix: rpc.jsonrpc - type: span - extends: rpc - brief: 'Tech-specific attributes for [JSON RPC](https://www.jsonrpc.org/).' - attributes: - - ref: rpc.jsonrpc.version - tag: jsonrpc-tech-specific - requirement_level: - conditionally_required: If other than the default version (`1.0`) - - ref: rpc.jsonrpc.request_id - tag: jsonrpc-tech-specific - - ref: rpc.jsonrpc.error_code - tag: jsonrpc-tech-specific - requirement_level: - conditionally_required: If response is not successful. - - ref: rpc.jsonrpc.error_message - tag: jsonrpc-tech-specific - - ref: rpc.method - tag: jsonrpc-tech-specific - requirement_level: required - note: > - This is always required for jsonrpc. See the note in the general - RPC conventions for more information. - - - id: rpc.message - prefix: "message" # TODO: Change the prefix to rpc.message? - type: event - brief: "RPC received/sent message." - attributes: - - id: type - type: - members: - - id: sent - value: "SENT" - - id: received - value: "RECEIVED" - brief: "Whether this is a received or sent message." - - id: id - type: int - brief: "MUST be calculated as two different counters starting from `1` one for sent messages and one for received message." - note: "This way we guarantee that the values will be consistent between different implementations." - - id: compressed_size - type: int - brief: "Compressed size of the message in bytes." - - id: uncompressed_size - type: int - brief: "Uncompressed size of the message in bytes." - - - id: rpc.connect_rpc - type: span - extends: rpc - brief: 'Tech-specific attributes for Connect RPC.' - attributes: - - ref: rpc.connect_rpc.error_code - tag: connect_rpc-tech-specific - requirement_level: - conditionally_required: If response is not successful and if error code available. - - ref: rpc.connect_rpc.request.metadata - tag: connect_rpc-tech-specific - requirement_level: opt_in - - ref: rpc.connect_rpc.response.metadata - tag: connect_rpc-tech-specific - requirement_level: opt_in diff --git a/internal/otelschema/_testdata/model/trace/trace-exception.yaml b/internal/otelschema/_testdata/model/trace/trace-exception.yaml deleted file mode 100644 index a11082eb..00000000 --- a/internal/otelschema/_testdata/model/trace/trace-exception.yaml +++ /dev/null @@ -1,38 +0,0 @@ -groups: - - id: trace-exception - prefix: exception - type: event - brief: > - This document defines the attributes used to - report a single exception associated with a span. - attributes: - - ref: exception.type - - ref: exception.message - - ref: exception.stacktrace - - id: escaped - type: boolean - brief: > - SHOULD be set to true if the exception event is recorded at a point where - it is known that the exception is escaping the scope of the span. - note: |- - An exception is considered to have escaped (or left) the scope of a span, - if that span is ended while the exception is still logically "in flight". - This may be actually "in flight" in some languages (e.g. if the exception - is passed to a Context manager's `__exit__` method in Python) but will - usually be caught at the point of recording the exception in most languages. - - It is usually not possible to determine at the point where an exception is thrown - whether it will escape the scope of a span. - However, it is trivial to know that an exception - will escape, if one checks for an active exception just before ending the span, - as done in the [example above](#recording-an-exception). - - It follows that an exception may still escape the scope of the span - even if the `exception.escaped` attribute was not set or set to false, - since the event might have been recorded at a time where it was not - clear whether the exception will escape. - - constraints: - - any_of: - - "exception.type" - - "exception.message" diff --git a/internal/otelschema/_testdata/model/url.yaml b/internal/otelschema/_testdata/model/url.yaml deleted file mode 100644 index bb27dc4e..00000000 --- a/internal/otelschema/_testdata/model/url.yaml +++ /dev/null @@ -1,13 +0,0 @@ -groups: - - id: url - brief: Attributes describing URL. - type: attribute_group - prefix: url - attributes: - - ref: url.scheme - - ref: url.full - tag: sensitive-information - - ref: url.path - - ref: url.query - tag: sensitive-information - - ref: url.fragment diff --git a/internal/otelschema/_testdata/model/version.properties b/internal/otelschema/_testdata/model/version.properties deleted file mode 100644 index 03770cf2..00000000 --- a/internal/otelschema/_testdata/model/version.properties +++ /dev/null @@ -1 +0,0 @@ -version=1 diff --git a/internal/otelschema/_testdata/semantic-conventions b/internal/otelschema/_testdata/semantic-conventions new file mode 160000 index 00000000..fe200c01 --- /dev/null +++ b/internal/otelschema/_testdata/semantic-conventions @@ -0,0 +1 @@ +Subproject commit fe200c01bcf589b30f2ae9b81152d69252d8ea21 diff --git a/internal/otelschema/group_test.go b/internal/otelschema/group_test.go index c52e5764..465b6d3c 100644 --- a/internal/otelschema/group_test.go +++ b/internal/otelschema/group_test.go @@ -10,8 +10,12 @@ import ( "sigs.k8s.io/yaml" ) +func modelPath() string { + return filepath.Join("_testdata", "semantic-conventions", "model") +} + func TestParse(t *testing.T) { - require.NoError(t, filepath.Walk(filepath.Join("_testdata", "model"), func(path string, info fs.FileInfo, err error) error { + require.NoError(t, filepath.Walk(modelPath(), func(path string, info fs.FileInfo, err error) error { require.NoError(t, err) if info.IsDir() { return nil diff --git a/internal/otelschema/list_test.go b/internal/otelschema/list_test.go deleted file mode 100644 index 3021b395..00000000 --- a/internal/otelschema/list_test.go +++ /dev/null @@ -1,85 +0,0 @@ -package otelschema - -import ( - "io/fs" - "os" - "path/filepath" - "testing" - - "github.com/go-faster/sdk/gold" - "github.com/stretchr/testify/require" - "sigs.k8s.io/yaml" -) - -func TestParseAllAttributes(t *testing.T) { - var parsed []TypeGroupsItem - require.NoError(t, filepath.Walk(filepath.Join("_testdata", "model"), func(path string, info fs.FileInfo, err error) error { - require.NoError(t, err) - if info.IsDir() { - return nil - } - if filepath.Ext(path) != ".yaml" { - return nil - } - data, err := os.ReadFile(path) - require.NoError(t, err) - - var schema Type - jsonData, err := yaml.YAMLToJSON(data) - require.NoError(t, err) - require.NoError(t, schema.UnmarshalJSON(jsonData)) - parsed = append(parsed, schema.Groups...) - return nil - })) - type entry struct { - Name string - Type string - Enum []interface{} `json:"Enum,omitempty"` - } - var entries []entry - for _, group := range parsed { - for _, attr := range group.Attributes { - v, ok := attr.GetAttribute() - if !ok { - continue - } - name := v.ID - if prefix, ok := group.Prefix.Get(); ok { - name = prefix + "." + name - } - var typ string - if s, ok := v.Type.GetString(); ok { - typ = s - } - var enum []interface{} - if e, ok := v.Type.GetEnum(); ok { - typ = "enum" - for _, m := range e.Members { - switch m.Value.Type { - case StringEnumMembersItemValue: - enum = append(enum, m.Value.String) - typ = "string" - case IntEnumMembersItemValue: - enum = append(enum, m.Value.Int) - typ = "int" - } - } - if e.AllowCustomValues.Value { - // Not actually an enum? - enum = nil - } - } - t.Logf("%s (%s)", name, typ) - entries = append(entries, entry{ - Name: name, - Type: typ, - Enum: enum, - }) - } - } - t.Logf("total: %d", len(entries)) - data, err := yaml.Marshal(entries) - require.NoError(t, err) - - gold.Str(t, string(data), "all_attributes.yaml") -} diff --git a/internal/otelschema/registry_test.go b/internal/otelschema/registry_test.go new file mode 100644 index 00000000..868d1c8d --- /dev/null +++ b/internal/otelschema/registry_test.go @@ -0,0 +1,208 @@ +package otelschema + +import ( + "fmt" + "io/fs" + "os" + "path/filepath" + "slices" + "strings" + "testing" + + "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/sdk/gold" + "github.com/stretchr/testify/require" + "sigs.k8s.io/yaml" +) + +type registryEntry struct { + Type string `json:"type"` + Enum []any `json:"enum,omitempty"` + Column proto.ColumnType `json:"column"` + Examples []any `json:"examples,omitempty"` + Brief string `json:"brief,omitempty"` +} + +func anyTo[T any](s []any) (result []T) { + result = make([]T, len(s)) + for i, v := range s { + result[i] = v.(T) + } + return result +} + +func columnType(name, t string, enum []any) proto.ColumnType { + isEnum := len(enum) > 0 + if !isEnum || t == "int" { + // We don't care about templates for storage data types. + if strings.HasPrefix(t, "template[") { + t = strings.TrimPrefix(t, "template[") + t = strings.TrimSuffix(t, "]") + } + colType := map[string]proto.ColumnType{ + "string": proto.ColumnTypeString, + "int": proto.ColumnTypeInt64, + "boolean": proto.ColumnTypeBool, + "string[]": proto.ColumnTypeString.Array(), + "int[]": proto.ColumnTypeInt64.Array(), + "double": proto.ColumnTypeFloat64, + }[t] + if strings.HasPrefix(name, "k8s.") && strings.HasSuffix(name, ".uid") { + // k8s.cluster.uid and others special case. + return proto.ColumnTypeUUID + } + if strings.HasSuffix(name, ".port") { + return proto.ColumnTypeUInt16 + } + + if isEnum { + // Handle enum of integers. + values := anyTo[int](enum) + maxValue := slices.Max(values) + minValue := slices.Min(values) + switch { + case minValue >= 0 && maxValue <= 255: + colType = proto.ColumnTypeUInt8 + case minValue >= 0 && maxValue <= 65535: + colType = proto.ColumnTypeUInt16 + case minValue < 0 && maxValue <= 127: + colType = proto.ColumnTypeInt8 + default: + colType = proto.ColumnTypeInt64 + } + } + if colType == "" { + return "UNKNOWN" + } + return colType + } + + // Handle enums. + colType := proto.ColumnTypeEnum8 + if len(enum) > 255 { + colType = proto.ColumnTypeEnum16 + } + var params []string + for i, v := range anyTo[string](enum) { + // Should we escape? + params = append(params, fmt.Sprintf("%d = '%s'", i, v)) + } + return colType.With(params...) +} + +func TestParseAllAttributes(t *testing.T) { + var parsed []TypeGroupsItem + require.NoError(t, filepath.Walk(modelPath(), func(path string, info fs.FileInfo, err error) error { + require.NoError(t, err) + if info.IsDir() { + return nil + } + if filepath.Ext(path) != ".yaml" { + return nil + } + data, err := os.ReadFile(path) + require.NoError(t, err) + + var schema Type + jsonData, err := yaml.YAMLToJSON(data) + require.NoError(t, err) + require.NoError(t, schema.UnmarshalJSON(jsonData)) + parsed = append(parsed, schema.Groups...) + return nil + })) + + type Statistics struct { + Total int `json:"total"` + Enum int `json:"enum"` + Unknown int `json:"unknown"` + } + type Registry struct { + Statistics Statistics `json:"statistics"` + Entries map[string]registryEntry `json:"entries"` + } + out := Registry{ + Entries: map[string]registryEntry{}, + } + for _, group := range parsed { + for _, attr := range group.Attributes { + v, ok := attr.GetAttribute() + if !ok { + continue + } + name := v.ID + if prefix, ok := group.Prefix.Get(); ok { + name = prefix + "." + name + } + var typ string + if s, ok := v.Type.GetString(); ok { + typ = s + } + var enum []interface{} + if e, ok := v.Type.GetEnum(); ok { + for _, m := range e.Members { + switch m.Value.Type { + case StringEnumMembersItemValue: + enum = append(enum, m.Value.String) + typ = "string" + case IntEnumMembersItemValue: + enum = append(enum, m.Value.Int) + typ = "int" + } + } + if e.AllowCustomValues.Value { + // Not actually an enum? + enum = nil + } + } + t.Logf("%s (%s)", name, typ) + var examples []any + + if e, ok := v.Examples.Get(); ok { + toAny := func(ev ExampleValue) any { + switch ev.Type { + case BoolExampleValue: + return ev.Bool + case Float64ExampleValue: + return ev.Float64 + case StringExampleValue: + return ev.String + } + return nil + } + for _, v := range e.ExampleValueArray { + examples = append(examples, toAny(v)) + } + if v, ok := e.GetExampleValue(); ok { + examples = append(examples, toAny(v)) + } + } + if len(enum) != 0 { + examples = nil + } + out.Entries[name] = registryEntry{ + Type: typ, + Enum: enum, + Column: columnType(name, typ, enum), + Examples: examples, + Brief: v.Brief.Value, + } + } + } + + t.Logf("total: %d", len(out.Entries)) + out.Statistics.Total = len(out.Entries) + for _, e := range out.Entries { + if e.Column == "UNKNOWN" { + out.Statistics.Unknown++ + } + if strings.HasPrefix(e.Column.String(), "Enum") { + out.Statistics.Enum++ + } + out.Statistics.Total++ + } + + data, err := yaml.Marshal(out) + require.NoError(t, err) + + gold.Str(t, string(data), "registry.yaml") +} From 83602c21eb0c494361a90bef5f06bac61adb4d40 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 14:50:30 +0300 Subject: [PATCH 055/112] feat(otelschema): handle deprecation --- internal/otelschema/_golden/registry.yaml | 150 +--------------------- internal/otelschema/registry_test.go | 14 +- 2 files changed, 13 insertions(+), 151 deletions(-) diff --git a/internal/otelschema/_golden/registry.yaml b/internal/otelschema/_golden/registry.yaml index cce2ccbd..cbb1cc92 100644 --- a/internal/otelschema/_golden/registry.yaml +++ b/internal/otelschema/_golden/registry.yaml @@ -989,18 +989,6 @@ entries: examples: - n1-standard-1 type: string - http.flavor: - brief: Deprecated, use `network.protocol.name` instead. - column: String - type: string - http.method: - brief: Deprecated, use `http.request.method` instead. - column: String - examples: - - GET - - POST - - HEAD - type: string http.request.body.size: brief: | The size of the request payload body in bytes. This is the number of bytes transferred excluding headers and is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) header. For requests using transport encoding, this should be the compressed size. @@ -1039,12 +1027,6 @@ entries: examples: - 3 type: int - http.request_content_length: - brief: Deprecated, use `http.request.header.content-length` instead. - column: Int64 - examples: - - 3495 - type: int http.response.body.size: brief: | The size of the response payload body in bytes. This is the number of bytes transferred excluding headers and is often, but not always, present as the [Content-Length](https://www.rfc-editor.org/rfc/rfc9110.html#field.content-length) header. For requests using transport encoding, this should be the compressed size. @@ -1066,12 +1048,6 @@ entries: examples: - 200 type: int - http.response_content_length: - brief: Deprecated, use `http.response.header.content-length` instead. - column: Int64 - examples: - - 3495 - type: int http.route: brief: | The matched route, that is, the path template in the format used by the respective server framework. @@ -1080,39 +1056,6 @@ entries: - /users/:userID? - '{controller}/{action}/{id?}' type: string - http.scheme: - brief: Deprecated, use `url.scheme` instead. - column: String - examples: - - http - - https - type: string - http.status_code: - brief: Deprecated, use `http.response.status_code` instead. - column: Int64 - examples: - - 200 - type: int - http.target: - brief: Deprecated, use `url.path` and `url.query` instead. - column: String - examples: - - /search?q=OpenTelemetry#SemConv - type: string - http.url: - brief: Deprecated, use `url.full` instead. - column: String - examples: - - https://www.foo.bar/search?q=OpenTelemetry#SemConv - type: string - http.user_agent: - brief: Deprecated, use `user_agent.original` instead. - column: String - examples: - - CERN-LineMode/2.15 libwww/2.17b3 - - Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 - (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1 - type: string ios.state: brief: | This attribute represents the state the application has transitioned into at the occurrence of the event. @@ -1592,84 +1535,6 @@ entries: An identifier for the messaging system being used. See below for a list of well-known identifiers. column: String type: string - net.host.name: - brief: Deprecated, use `server.address`. - column: String - examples: - - example.com - type: string - net.host.port: - brief: Deprecated, use `server.port`. - column: UInt16 - examples: - - 8080 - type: int - net.peer.name: - brief: Deprecated, use `server.address` on client spans and `client.address` on - server spans. - column: String - examples: - - example.com - type: string - net.peer.port: - brief: Deprecated, use `server.port` on client spans and `client.port` on server - spans. - column: UInt16 - examples: - - 8080 - type: int - net.protocol.name: - brief: Deprecated, use `network.protocol.name`. - column: String - examples: - - amqp - - http - - mqtt - type: string - net.protocol.version: - brief: Deprecated, use `network.protocol.version`. - column: String - examples: - - 3.1.1 - type: string - net.sock.family: - brief: Deprecated, use `network.transport` and `network.type`. - column: String - type: string - net.sock.host.addr: - brief: Deprecated, use `network.local.address`. - column: String - examples: - - /var/my.sock - type: string - net.sock.host.port: - brief: Deprecated, use `network.local.port`. - column: UInt16 - examples: - - 8080 - type: int - net.sock.peer.addr: - brief: Deprecated, use `network.peer.address`. - column: String - examples: - - 192.168.0.1 - type: string - net.sock.peer.name: - brief: Deprecated, no replacement at this time. - column: String - examples: - - /var/my.sock - type: string - net.sock.peer.port: - brief: Deprecated, use `network.peer.port`. - column: UInt16 - examples: - - 65531 - type: int - net.transport: - brief: Deprecated, use `network.transport`. - column: String - type: string network.carrier.icc: brief: The ISO 3166-1 alpha-2 2-character country code associated with the mobile carrier network. @@ -1826,18 +1691,6 @@ entries: - 14.2.1 - 18.04.1 type: string - otel.library.name: - brief: Deprecated, use the `otel.scope.name` attribute. - column: String - examples: - - io.opentelemetry.contrib.mongodb - type: string - otel.library.version: - brief: Deprecated, use the `otel.scope.version` attribute. - column: String - examples: - - 1.0.0 - type: string otel.scope.name: brief: The name of the instrumentation scope - (`InstrumentationScope.Name` in OTLP). @@ -2576,6 +2429,7 @@ entries: - 21.0.0 type: string statistics: + deprecated: 24 enum: 23 - total: 716 + total: 668 unknown: 0 diff --git a/internal/otelschema/registry_test.go b/internal/otelschema/registry_test.go index 868d1c8d..d21c4d27 100644 --- a/internal/otelschema/registry_test.go +++ b/internal/otelschema/registry_test.go @@ -11,6 +11,7 @@ import ( "github.com/ClickHouse/ch-go/proto" "github.com/go-faster/sdk/gold" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "sigs.k8s.io/yaml" ) @@ -112,9 +113,10 @@ func TestParseAllAttributes(t *testing.T) { })) type Statistics struct { - Total int `json:"total"` - Enum int `json:"enum"` - Unknown int `json:"unknown"` + Total int `json:"total"` + Enum int `json:"enum"` + Unknown int `json:"unknown"` + Deprecated int `json:"deprecated"` } type Registry struct { Statistics Statistics `json:"statistics"` @@ -125,6 +127,10 @@ func TestParseAllAttributes(t *testing.T) { } for _, group := range parsed { for _, attr := range group.Attributes { + if attr.Attribute.Stability.Value == "deprecated" { + out.Statistics.Deprecated++ + continue + } v, ok := attr.GetAttribute() if !ok { continue @@ -205,4 +211,6 @@ func TestParseAllAttributes(t *testing.T) { require.NoError(t, err) gold.Str(t, string(data), "registry.yaml") + + assert.Zero(t, out.Statistics.Unknown, "Should be no unknown types") } From 26988288e757dec37754fe11f77ec77dce6966e2 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 14:55:04 +0300 Subject: [PATCH 056/112] feat(otelschema): handle ISO 8601 --- internal/otelschema/_golden/registry.yaml | 4 ++-- internal/otelschema/registry_test.go | 8 ++++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/internal/otelschema/_golden/registry.yaml b/internal/otelschema/_golden/registry.yaml index cbb1cc92..66fe442c 100644 --- a/internal/otelschema/_golden/registry.yaml +++ b/internal/otelschema/_golden/registry.yaml @@ -732,7 +732,7 @@ entries: faas.document.time: brief: | A string containing the time when the data was accessed in the [ISO 8601](https://www.iso.org/iso-8601-date-and-time-format.html) format expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). - column: String + column: DateTime examples: - "2020-01-23T13:47:06Z" type: string @@ -786,7 +786,7 @@ entries: faas.time: brief: | A string containing the function invocation time in the [ISO 8601](https://www.iso.org/iso-8601-date-and-time-format.html) format expressed in [UTC](https://www.w3.org/TR/NOTE-datetime). - column: String + column: DateTime examples: - "2020-01-23T13:47:06Z" type: string diff --git a/internal/otelschema/registry_test.go b/internal/otelschema/registry_test.go index d21c4d27..868d3ed0 100644 --- a/internal/otelschema/registry_test.go +++ b/internal/otelschema/registry_test.go @@ -32,7 +32,7 @@ func anyTo[T any](s []any) (result []T) { return result } -func columnType(name, t string, enum []any) proto.ColumnType { +func columnType(name, brief, t string, enum []any) proto.ColumnType { isEnum := len(enum) > 0 if !isEnum || t == "int" { // We don't care about templates for storage data types. @@ -55,6 +55,10 @@ func columnType(name, t string, enum []any) proto.ColumnType { if strings.HasSuffix(name, ".port") { return proto.ColumnTypeUInt16 } + if strings.Contains(brief, "ISO 8601") && t == "string" { + // Just seconds? + return proto.ColumnTypeDateTime + } if isEnum { // Handle enum of integers. @@ -188,7 +192,7 @@ func TestParseAllAttributes(t *testing.T) { out.Entries[name] = registryEntry{ Type: typ, Enum: enum, - Column: columnType(name, typ, enum), + Column: columnType(name, v.Brief.Value, typ, enum), Examples: examples, Brief: v.Brief.Value, } From 0280abc47fba441a3dbbcf2866a8b8a336bf8424 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 15:00:07 +0300 Subject: [PATCH 057/112] ci: submodules --- .github/workflows/e2e.yml | 2 ++ .github/workflows/x.yml | 3 +++ 2 files changed, 5 insertions(+) diff --git a/.github/workflows/e2e.yml b/.github/workflows/e2e.yml index b151e1c5..ed37cbb8 100644 --- a/.github/workflows/e2e.yml +++ b/.github/workflows/e2e.yml @@ -12,6 +12,8 @@ jobs: steps: - name: Checkout code uses: actions/checkout@v4 + with: + submodules: true - name: Install Go uses: actions/setup-go@v4 diff --git a/.github/workflows/x.yml b/.github/workflows/x.yml index 4a7489b0..907aa2aa 100644 --- a/.github/workflows/x.yml +++ b/.github/workflows/x.yml @@ -13,7 +13,10 @@ jobs: uses: go-faster/x/.github/workflows/test.yml@main with: enable-386: false + submodules: true cover: uses: go-faster/x/.github/workflows/cover.yml@main + with: + submodules: true lint: uses: go-faster/x/.github/workflows/lint.yml@main From b4c0d6ca8833b0e69683b8f1e66fb0332f2c8773 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 15:04:34 +0300 Subject: [PATCH 058/112] feat(otelschema): use underscores --- internal/otelschema/_golden/registry.yaml | 334 ++++++++++++++++++++++ internal/otelschema/registry_test.go | 2 + 2 files changed, 336 insertions(+) diff --git a/internal/otelschema/_golden/registry.yaml b/internal/otelschema/_golden/registry.yaml index 66fe442c..715b010c 100644 --- a/internal/otelschema/_golden/registry.yaml +++ b/internal/otelschema/_golden/registry.yaml @@ -6,6 +6,7 @@ entries: examples: - "33" - "32" + name: android_os_api_level type: string android.state: brief: | @@ -15,6 +16,7 @@ entries: - created - background - foreground + name: android_state type: string aws.ecs.cluster.arn: brief: | @@ -22,6 +24,7 @@ entries: column: String examples: - arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster + name: aws_ecs_cluster_arn type: string aws.ecs.container.arn: brief: | @@ -29,6 +32,7 @@ entries: column: String examples: - arn:aws:ecs:us-west-1:123456789123:container/32624152-9086-4f0e-acae-1a75b14fe4d9 + name: aws_ecs_container_arn type: string aws.ecs.launchtype: brief: | @@ -37,6 +41,7 @@ entries: enum: - ec2 - fargate + name: aws_ecs_launchtype type: string aws.ecs.task.arn: brief: | @@ -44,6 +49,7 @@ entries: column: String examples: - arn:aws:ecs:us-west-1:123456789123:task/10838bed-421f-43ef-870a-f43feacbbb5b + name: aws_ecs_task_arn type: string aws.ecs.task.family: brief: | @@ -51,6 +57,7 @@ entries: column: String examples: - opentelemetry-family + name: aws_ecs_task_family type: string aws.ecs.task.revision: brief: | @@ -59,6 +66,7 @@ entries: examples: - "8" - "26" + name: aws_ecs_task_revision type: string aws.eks.cluster.arn: brief: | @@ -66,6 +74,7 @@ entries: column: String examples: - arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster + name: aws_eks_cluster_arn type: string aws.lambda.invoked_arn: brief: | @@ -73,6 +82,7 @@ entries: column: String examples: - arn:aws:lambda:us-east-1:123456:function:myfunction:myalias + name: aws_lambda_invoked_arn type: string aws.log.group.arns: brief: | @@ -80,6 +90,7 @@ entries: column: Array(String) examples: - arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:* + name: aws_log_group_arns type: string[] aws.log.group.names: brief: | @@ -88,6 +99,7 @@ entries: examples: - /aws/lambda/my-function - opentelemetry-service + name: aws_log_group_names type: string[] aws.log.stream.arns: brief: | @@ -95,6 +107,7 @@ entries: column: Array(String) examples: - arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:log-stream:logs/main/10838bed-421f-43ef-870a-f43feacbbb5b + name: aws_log_stream_arns type: string[] aws.log.stream.names: brief: | @@ -102,6 +115,7 @@ entries: column: Array(String) examples: - logs/main/10838bed-421f-43ef-870a-f43feacbbb5b + name: aws_log_stream_names type: string[] browser.brands: brief: Array of brand name and version separated by a space @@ -110,6 +124,7 @@ entries: - ' Not A;Brand 99' - Chromium 99 - Chrome 99 + name: browser_brands type: string[] browser.language: brief: Preferred language of the user using the browser @@ -119,10 +134,12 @@ entries: - en-US - fr - fr-FR + name: browser_language type: string browser.mobile: brief: A boolean that is true if the browser is running on a mobile device column: Bool + name: browser_mobile type: boolean browser.platform: brief: The platform on which the browser is running @@ -131,6 +148,7 @@ entries: - Windows - macOS - Android + name: browser_platform type: string client.address: brief: Client address - domain name if available without reverse DNS lookup; otherwise, @@ -140,12 +158,14 @@ entries: - client.example.com - 10.1.2.80 - /tmp/my.sock + name: client_address type: string client.port: brief: Client port number. column: UInt16 examples: - 65123 + name: client_port type: int cloud.account.id: brief: | @@ -154,6 +174,7 @@ entries: examples: - "111111111111" - opentelemetry + name: cloud_account_id type: string cloud.availability_zone: brief: | @@ -161,16 +182,19 @@ entries: column: String examples: - us-east-1c + name: cloud_availability_zone type: string cloud.platform: brief: | The cloud platform in use. column: String + name: cloud_platform type: string cloud.provider: brief: | Name of the cloud provider. column: String + name: cloud_provider type: string cloud.region: brief: | @@ -179,6 +203,7 @@ entries: examples: - us-central1 - us-east-1 + name: cloud_region type: string cloud.resource_id: brief: | @@ -188,6 +213,7 @@ entries: - arn:aws:lambda:REGION:ACCOUNT_ID:function:my-function - //run.googleapis.com/projects/PROJECT_ID/locations/LOCATION_ID/services/SERVICE_ID - /subscriptions//resourceGroups//providers/Microsoft.Web/sites//functions/ + name: cloud_resource_id type: string cloudevents.event_id: brief: | @@ -196,6 +222,7 @@ entries: examples: - 123e4567-e89b-12d3-a456-426614174000 - "0001" + name: cloudevents_event_id type: string cloudevents.event_source: brief: | @@ -205,6 +232,7 @@ entries: - https://github.com/cloudevents - /cloudevents/spec/pull/123 - my-service + name: cloudevents_event_source type: string cloudevents.event_spec_version: brief: | @@ -212,6 +240,7 @@ entries: column: String examples: - "1.0" + name: cloudevents_event_spec_version type: string cloudevents.event_subject: brief: | @@ -219,6 +248,7 @@ entries: column: String examples: - mynewfile.jpg + name: cloudevents_event_subject type: string cloudevents.event_type: brief: | @@ -227,6 +257,7 @@ entries: examples: - com.github.pull_request.opened - com.example.object.deleted.v2 + name: cloudevents_event_type type: string code.column: brief: | @@ -234,6 +265,7 @@ entries: column: Int64 examples: - 16 + name: code_column type: int code.filepath: brief: | @@ -241,6 +273,7 @@ entries: column: String examples: - /usr/local/MyApplication/content_root/app/index.php + name: code_filepath type: string code.function: brief: | @@ -248,6 +281,7 @@ entries: column: String examples: - serveRequest + name: code_function type: string code.lineno: brief: | @@ -255,6 +289,7 @@ entries: column: Int64 examples: - 42 + name: code_lineno type: int code.namespace: brief: | @@ -262,6 +297,7 @@ entries: column: String examples: - com.example.MyHttpService + name: code_namespace type: string code.stacktrace: brief: | @@ -270,6 +306,7 @@ entries: examples: - at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n at com.example.GenerateTrace.main(GenerateTrace.java:5) + name: code_stacktrace type: string container.command: brief: | @@ -277,6 +314,7 @@ entries: column: String examples: - otelcontribcol + name: container_command type: string container.command_args: brief: | @@ -284,6 +322,7 @@ entries: column: Array(String) examples: - otelcontribcol, --config, config.yaml + name: container_command_args type: string[] container.command_line: brief: | @@ -291,6 +330,7 @@ entries: column: String examples: - otelcontribcol --config config.yaml + name: container_command_line type: string container.id: brief: | @@ -298,6 +338,7 @@ entries: column: String examples: - a3bf90e006b2 + name: container_id type: string container.image.id: brief: | @@ -305,6 +346,7 @@ entries: column: String examples: - sha256:19c92d0a00d1b66d897bceaa7319bee0dd38a10a851c60bcec9474aa3f01e50f + name: container_image_id type: string container.image.name: brief: | @@ -312,6 +354,7 @@ entries: column: String examples: - gcr.io/opentelemetry/operator + name: container_image_name type: string container.image.repo_digests: brief: | @@ -320,6 +363,7 @@ entries: examples: - example@sha256:afcc7f1ac1b49db317a7196c902e61c6c3c4607d63599ee1a82d702d249a0ccb - internal.registry.example.com:5000/example@sha256:b69959407d21e8a062e0416bf13405bb2b71ed7a84dde4158ebafacfa06f5578 + name: container_image_repo_digests type: string[] container.image.tags: brief: | @@ -328,6 +372,7 @@ entries: examples: - v1.27.1 - 3.5.7-0 + name: container_image_tags type: string[] container.labels: brief: | @@ -335,6 +380,7 @@ entries: column: String examples: - container.labels.app=nginx + name: container_labels type: template[string] container.name: brief: | @@ -342,6 +388,7 @@ entries: column: String examples: - opentelemetry-autoconf + name: container_name type: string container.runtime: brief: | @@ -351,6 +398,7 @@ entries: - docker - containerd - rkt + name: container_runtime type: string db.cassandra.consistency_level: brief: | @@ -370,6 +418,7 @@ entries: - any - serial - local_serial + name: db_cassandra_consistency_level type: string db.cassandra.coordinator.dc: brief: | @@ -377,6 +426,7 @@ entries: column: String examples: - us-west-2 + name: db_cassandra_coordinator_dc type: string db.cassandra.coordinator.id: brief: | @@ -384,11 +434,13 @@ entries: column: String examples: - be13faa2-8574-4d71-926d-27f16cf8a7af + name: db_cassandra_coordinator_id type: string db.cassandra.idempotence: brief: | Whether or not the query is idempotent. column: Bool + name: db_cassandra_idempotence type: boolean db.cassandra.page_size: brief: | @@ -396,6 +448,7 @@ entries: column: Int64 examples: - 5000 + name: db_cassandra_page_size type: int db.cassandra.speculative_execution_count: brief: | @@ -404,6 +457,7 @@ entries: examples: - 0 - 2 + name: db_cassandra_speculative_execution_count type: int db.cassandra.table: brief: The name of the primary Cassandra table that the operation is acting upon, @@ -411,6 +465,7 @@ entries: column: String examples: - mytable + name: db_cassandra_table type: string db.connection_string: brief: | @@ -418,12 +473,14 @@ entries: column: String examples: - Server=(localdb)\v11.0;Integrated Security=true; + name: db_connection_string type: string db.cosmosdb.client_id: brief: Unique Cosmos client instance id. column: String examples: - 3ba4827d-4422-483f-b59f-85b74211c11d + name: db_cosmosdb_client_id type: string db.cosmosdb.connection_mode: brief: Cosmos client connection mode. @@ -431,16 +488,19 @@ entries: enum: - gateway - direct + name: db_cosmosdb_connection_mode type: string db.cosmosdb.container: brief: Cosmos DB container name. column: String examples: - anystring + name: db_cosmosdb_container type: string db.cosmosdb.operation_type: brief: CosmosDB Operation Type. column: String + name: db_cosmosdb_operation_type type: string db.cosmosdb.request_charge: brief: RU consumed for that operation @@ -448,10 +508,12 @@ entries: examples: - 46.18 - 1 + name: db_cosmosdb_request_charge type: double db.cosmosdb.request_content_length: brief: Request payload size in bytes column: Int64 + name: db_cosmosdb_request_content_length type: int db.cosmosdb.status_code: brief: Cosmos DB status code. @@ -459,6 +521,7 @@ entries: examples: - 200 - 201 + name: db_cosmosdb_status_code type: int db.cosmosdb.sub_status_code: brief: Cosmos DB sub status code. @@ -466,6 +529,7 @@ entries: examples: - 1000 - 1002 + name: db_cosmosdb_sub_status_code type: int db.elasticsearch.cluster.name: brief: | @@ -473,6 +537,7 @@ entries: column: String examples: - e9106fc68e3044f0b1475b04bf4ffd5f + name: db_elasticsearch_cluster_name type: string db.elasticsearch.node.name: brief: | @@ -480,6 +545,7 @@ entries: column: String examples: - instance-0000000001 + name: db_elasticsearch_node_name type: string db.elasticsearch.path_parts: brief: | @@ -488,6 +554,7 @@ entries: examples: - db.elasticsearch.path_parts.index=test-index - db.elasticsearch.path_parts.doc_id=123 + name: db_elasticsearch_path_parts type: template[string] db.jdbc.driver_classname: brief: | @@ -496,6 +563,7 @@ entries: examples: - org.postgresql.Driver - com.microsoft.sqlserver.jdbc.SQLServerDriver + name: db_jdbc_driver_classname type: string db.mongodb.collection: brief: | @@ -504,6 +572,7 @@ entries: examples: - customers - products + name: db_mongodb_collection type: string db.mssql.instance_name: brief: | @@ -511,6 +580,7 @@ entries: column: String examples: - MSSQLSERVER + name: db_mssql_instance_name type: string db.name: brief: | @@ -519,6 +589,7 @@ entries: examples: - customers - main + name: db_name type: string db.operation: brief: | @@ -528,6 +599,7 @@ entries: - findAndModify - HMSET - SELECT + name: db_operation type: string db.redis.database_index: brief: | @@ -537,6 +609,7 @@ entries: - 0 - 1 - 15 + name: db_redis_database_index type: int db.sql.table: brief: The name of the primary table that the operation is acting upon, including @@ -545,6 +618,7 @@ entries: examples: - public.users - customers + name: db_sql_table type: string db.statement: brief: | @@ -553,11 +627,13 @@ entries: examples: - SELECT * FROM wuser_table - SET mykey "WuValue" + name: db_statement type: string db.system: brief: An identifier for the database management system (DBMS) product being used. See below for a list of well-known identifiers. column: String + name: db_system type: string db.user: brief: | @@ -566,6 +642,7 @@ entries: examples: - readonly_user - reporting_user + name: db_user type: string deployment.environment: brief: | @@ -574,6 +651,7 @@ entries: examples: - staging - production + name: deployment_environment type: string destination.address: brief: Destination address - domain name if available without reverse DNS lookup; @@ -583,6 +661,7 @@ entries: - destination.example.com - 10.1.2.80 - /tmp/my.sock + name: destination_address type: string destination.port: brief: Destination port number @@ -590,6 +669,7 @@ entries: examples: - 3389 - 2888 + name: destination_port type: int device.id: brief: | @@ -597,6 +677,7 @@ entries: column: String examples: - 2ab2916d-a51f-4ac8-80ee-45ac31a28092 + name: device_id type: string device.manufacturer: brief: | @@ -605,6 +686,7 @@ entries: examples: - Apple - Samsung + name: device_manufacturer type: string device.model.identifier: brief: | @@ -613,6 +695,7 @@ entries: examples: - iPhone3,4 - SM-G920F + name: device_model_identifier type: string device.model.name: brief: | @@ -621,6 +704,7 @@ entries: examples: - iPhone 6s Plus - Samsung Galaxy S6 + name: device_model_name type: string disk.io.direction: brief: The disk IO operation direction. @@ -628,6 +712,7 @@ entries: enum: - read - write + name: disk_io_direction type: string enduser.id: brief: | @@ -635,6 +720,7 @@ entries: column: String examples: - username + name: enduser_id type: string enduser.role: brief: Actual/assumed role the client is making the request under extracted from @@ -642,6 +728,7 @@ entries: column: String examples: - admin + name: enduser_role type: string enduser.scope: brief: | @@ -649,6 +736,7 @@ entries: column: String examples: - read:message, write:files + name: enduser_scope type: string error.type: brief: | @@ -659,6 +747,7 @@ entries: - java.net.UnknownHostException - server_certificate_invalid - "500" + name: error_type type: string event.name: brief: | @@ -667,11 +756,13 @@ entries: examples: - browser.mouse.click - device.app.lifecycle + name: event_name type: string exception.escaped: brief: | SHOULD be set to true if the exception event is recorded at a point where it is known that the exception is escaping the scope of the span. column: Bool + name: exception_escaped type: boolean exception.message: brief: The exception message. @@ -679,6 +770,7 @@ entries: examples: - Division by zero - Can't convert 'int' object to str implicitly + name: exception_message type: string exception.stacktrace: brief: | @@ -688,6 +780,7 @@ entries: - 'Exception in thread "main" java.lang.RuntimeException: Test exception\n at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n at com.example.GenerateTrace.main(GenerateTrace.java:5)' + name: exception_stacktrace type: string exception.type: brief: | @@ -696,11 +789,13 @@ entries: examples: - java.net.ConnectException - OSError + name: exception_type type: string faas.coldstart: brief: | A boolean that is true if the serverless function is executed for the first time (aka cold-start). column: Bool + name: faas_coldstart type: boolean faas.cron: brief: | @@ -708,6 +803,7 @@ entries: column: String examples: - 0/5 * * * ? * + name: faas_cron type: string faas.document.collection: brief: | @@ -716,6 +812,7 @@ entries: examples: - myBucketName - myDbName + name: faas_document_collection type: string faas.document.name: brief: | @@ -724,10 +821,12 @@ entries: examples: - myFile.txt - myTableName + name: faas_document_name type: string faas.document.operation: brief: Describes the type of the operation that was performed on the data. column: String + name: faas_document_operation type: string faas.document.time: brief: | @@ -735,6 +834,7 @@ entries: column: DateTime examples: - "2020-01-23T13:47:06Z" + name: faas_document_time type: string faas.instance: brief: | @@ -742,12 +842,14 @@ entries: column: String examples: - 2021/06/28/[$LATEST]2f399eb14537447da05ab2a2e39309de + name: faas_instance type: string faas.invocation_id: brief: The invocation ID of the current function invocation. column: String examples: - af9d5aa4-a685-4c5f-a22b-444f80b3cc28 + name: faas_invocation_id type: string faas.invoked_name: brief: | @@ -755,11 +857,13 @@ entries: column: String examples: - my-function + name: faas_invoked_name type: string faas.invoked_provider: brief: | The cloud provider of the invoked function. column: String + name: faas_invoked_provider type: string faas.invoked_region: brief: | @@ -767,6 +871,7 @@ entries: column: String examples: - eu-central-1 + name: faas_invoked_region type: string faas.max_memory: brief: | @@ -774,6 +879,7 @@ entries: column: Int64 examples: - 134217728 + name: faas_max_memory type: int faas.name: brief: | @@ -782,6 +888,7 @@ entries: examples: - my-function - myazurefunctionapp/some-function-name + name: faas_name type: string faas.time: brief: | @@ -789,6 +896,7 @@ entries: column: DateTime examples: - "2020-01-23T13:47:06Z" + name: faas_time type: string faas.trigger: brief: Type of the trigger which caused this function invocation. @@ -799,6 +907,7 @@ entries: - pubsub - timer - other + name: faas_trigger type: string faas.version: brief: The immutable version of the function being executed. @@ -806,18 +915,21 @@ entries: examples: - "26" - pinkfroid-00002 + name: faas_version type: string feature_flag.key: brief: The unique identifier of the feature flag. column: String examples: - logo-color + name: feature_flag_key type: string feature_flag.provider_name: brief: The name of the service provider that performs the flag evaluation. column: String examples: - Flag Manager + name: feature_flag_provider_name type: string feature_flag.variant: brief: | @@ -827,6 +939,7 @@ entries: - red - "true" - "on" + name: feature_flag_variant type: string gcp.cloud_run.job.execution: brief: | @@ -835,6 +948,7 @@ entries: examples: - job-name-xxxx - sample-job-mdw84 + name: gcp_cloud_run_job_execution type: string gcp.cloud_run.job.task_index: brief: | @@ -843,6 +957,7 @@ entries: examples: - 0 - 1 + name: gcp_cloud_run_job_task_index type: int gcp.gce.instance.hostname: brief: | @@ -851,6 +966,7 @@ entries: examples: - my-host1234.example.com - sample-vm.us-west1-b.c.my-project.internal + name: gcp_gce_instance_hostname type: string gcp.gce.instance.name: brief: | @@ -859,6 +975,7 @@ entries: examples: - instance-1 - my-vm-name + name: gcp_gce_instance_name type: string heroku.app.id: brief: | @@ -866,6 +983,7 @@ entries: column: String examples: - 2daa2797-e42b-4624-9322-ec3f968df4da + name: heroku_app_id type: string heroku.release.commit: brief: | @@ -873,6 +991,7 @@ entries: column: String examples: - e6134959463efd8966b20e75b913cafe3f5ec + name: heroku_release_commit type: string heroku.release.creation_timestamp: brief: | @@ -880,11 +999,13 @@ entries: column: String examples: - "2022-10-23T18:00:42Z" + name: heroku_release_creation_timestamp type: string host.arch: brief: | The CPU architecture the host system is running on. column: String + name: host_arch type: string host.cpu.cache.l2.size: brief: | @@ -892,6 +1013,7 @@ entries: column: Int64 examples: - 12288000 + name: host_cpu_cache_l2_size type: int host.cpu.family: brief: | @@ -900,6 +1022,7 @@ entries: examples: - "6" - PA-RISC 1.1e + name: host_cpu_family type: string host.cpu.model.id: brief: | @@ -908,6 +1031,7 @@ entries: examples: - "6" - 9000/778/B180L + name: host_cpu_model_id type: string host.cpu.model.name: brief: | @@ -915,6 +1039,7 @@ entries: column: String examples: - 11th Gen Intel(R) Core(TM) i7-1185G7 @ 3.00GHz + name: host_cpu_model_name type: string host.cpu.stepping: brief: | @@ -922,6 +1047,7 @@ entries: column: Int64 examples: - 1 + name: host_cpu_stepping type: int host.cpu.vendor.id: brief: | @@ -929,6 +1055,7 @@ entries: column: String examples: - GenuineIntel + name: host_cpu_vendor_id type: string host.id: brief: | @@ -936,6 +1063,7 @@ entries: column: String examples: - fdbf79e8af94cb7f9e8df36789187052 + name: host_id type: string host.image.id: brief: | @@ -943,6 +1071,7 @@ entries: column: String examples: - ami-07b06b442921831e5 + name: host_image_id type: string host.image.name: brief: | @@ -951,6 +1080,7 @@ entries: examples: - infra-ami-eks-worker-node-7d4ec78312 - CentOS-8-x86_64-1905 + name: host_image_name type: string host.image.version: brief: | @@ -958,6 +1088,7 @@ entries: column: String examples: - "0.1" + name: host_image_version type: string host.ip: brief: | @@ -966,6 +1097,7 @@ entries: examples: - 192.168.1.140 - fe80::abc2:4a28:737a:609e + name: host_ip type: string[] host.mac: brief: | @@ -974,6 +1106,7 @@ entries: examples: - AC-DE-48-23-45-67 - AC-DE-48-23-45-67-01-9F + name: host_mac type: string[] host.name: brief: | @@ -981,6 +1114,7 @@ entries: column: String examples: - opentelemetry-test + name: host_name type: string host.type: brief: | @@ -988,6 +1122,7 @@ entries: column: String examples: - n1-standard-1 + name: host_type type: string http.request.body.size: brief: | @@ -995,6 +1130,7 @@ entries: column: Int64 examples: - 3495 + name: http_request_body_size type: int http.request.header: brief: | @@ -1003,6 +1139,7 @@ entries: examples: - http.request.header.content-type=["application/json"] - http.request.header.x-forwarded-for=["1.2.3.4", "1.2.3.5"] + name: http_request_header type: template[string[]] http.request.method: brief: HTTP request method. @@ -1011,6 +1148,7 @@ entries: - GET - POST - HEAD + name: http_request_method type: string http.request.method_original: brief: Original HTTP method sent by the client in the request line. @@ -1019,6 +1157,7 @@ entries: - GeT - ACL - foo + name: http_request_method_original type: string http.request.resend_count: brief: | @@ -1026,6 +1165,7 @@ entries: column: Int64 examples: - 3 + name: http_request_resend_count type: int http.response.body.size: brief: | @@ -1033,6 +1173,7 @@ entries: column: Int64 examples: - 3495 + name: http_response_body_size type: int http.response.header: brief: | @@ -1041,12 +1182,14 @@ entries: examples: - http.response.header.content-type=["application/json"] - http.response.header.my-custom-header=["abc", "def"] + name: http_response_header type: template[string[]] http.response.status_code: brief: '[HTTP response status code](https://tools.ietf.org/html/rfc7231#section-6).' column: Int64 examples: - 200 + name: http_response_status_code type: int http.route: brief: | @@ -1055,6 +1198,7 @@ entries: examples: - /users/:userID? - '{controller}/{action}/{id?}' + name: http_route type: string ios.state: brief: | @@ -1067,6 +1211,7 @@ entries: - background - foreground - terminate + name: ios_state type: string jvm.buffer.pool.name: brief: Name of the buffer pool. @@ -1074,6 +1219,7 @@ entries: examples: - mapped - direct + name: jvm_buffer_pool_name type: string jvm.gc.action: brief: Name of the garbage collector action. @@ -1081,6 +1227,7 @@ entries: examples: - end of minor GC - end of major GC + name: jvm_gc_action type: string jvm.gc.name: brief: Name of the garbage collector. @@ -1088,6 +1235,7 @@ entries: examples: - G1 Young Generation - G1 Old Generation + name: jvm_gc_name type: string jvm.memory.pool.name: brief: Name of the memory pool. @@ -1096,6 +1244,7 @@ entries: - G1 Old Gen - G1 Eden space - G1 Survivor Space + name: jvm_memory_pool_name type: string jvm.memory.type: brief: The type of memory. @@ -1103,10 +1252,12 @@ entries: enum: - heap - non_heap + name: jvm_memory_type type: string jvm.thread.daemon: brief: Whether the thread is daemon or not. column: Bool + name: jvm_thread_daemon type: boolean jvm.thread.state: brief: State of the thread. @@ -1119,6 +1270,7 @@ entries: - waiting - timed_waiting - terminated + name: jvm_thread_state type: string k8s.cluster.name: brief: | @@ -1126,6 +1278,7 @@ entries: column: String examples: - opentelemetry-cluster + name: k8s_cluster_name type: string k8s.cluster.uid: brief: | @@ -1133,6 +1286,7 @@ entries: column: UUID examples: - 218fc5a9-a5f1-4b54-aa05-46717d0ab26d + name: k8s_cluster_uid type: string k8s.container.name: brief: | @@ -1140,6 +1294,7 @@ entries: column: String examples: - redis + name: k8s_container_name type: string k8s.container.restart_count: brief: | @@ -1148,6 +1303,7 @@ entries: examples: - 0 - 2 + name: k8s_container_restart_count type: int k8s.cronjob.name: brief: | @@ -1155,6 +1311,7 @@ entries: column: String examples: - opentelemetry + name: k8s_cronjob_name type: string k8s.cronjob.uid: brief: | @@ -1162,6 +1319,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + name: k8s_cronjob_uid type: string k8s.daemonset.name: brief: | @@ -1169,6 +1327,7 @@ entries: column: String examples: - opentelemetry + name: k8s_daemonset_name type: string k8s.daemonset.uid: brief: | @@ -1176,6 +1335,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + name: k8s_daemonset_uid type: string k8s.deployment.name: brief: | @@ -1183,6 +1343,7 @@ entries: column: String examples: - opentelemetry + name: k8s_deployment_name type: string k8s.deployment.uid: brief: | @@ -1190,6 +1351,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + name: k8s_deployment_uid type: string k8s.job.name: brief: | @@ -1197,6 +1359,7 @@ entries: column: String examples: - opentelemetry + name: k8s_job_name type: string k8s.job.uid: brief: | @@ -1204,6 +1367,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + name: k8s_job_uid type: string k8s.namespace.name: brief: | @@ -1211,6 +1375,7 @@ entries: column: String examples: - default + name: k8s_namespace_name type: string k8s.node.name: brief: | @@ -1218,6 +1383,7 @@ entries: column: String examples: - node-1 + name: k8s_node_name type: string k8s.node.uid: brief: | @@ -1225,6 +1391,7 @@ entries: column: UUID examples: - 1eb3a0c6-0477-4080-a9cb-0cb7db65c6a2 + name: k8s_node_uid type: string k8s.pod.name: brief: | @@ -1232,6 +1399,7 @@ entries: column: String examples: - opentelemetry-pod-autoconf + name: k8s_pod_name type: string k8s.pod.uid: brief: | @@ -1239,6 +1407,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + name: k8s_pod_uid type: string k8s.replicaset.name: brief: | @@ -1246,6 +1415,7 @@ entries: column: String examples: - opentelemetry + name: k8s_replicaset_name type: string k8s.replicaset.uid: brief: | @@ -1253,6 +1423,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + name: k8s_replicaset_uid type: string k8s.statefulset.name: brief: | @@ -1260,6 +1431,7 @@ entries: column: String examples: - opentelemetry + name: k8s_statefulset_name type: string k8s.statefulset.uid: brief: | @@ -1267,6 +1439,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + name: k8s_statefulset_uid type: string log.file.name: brief: | @@ -1274,6 +1447,7 @@ entries: column: String examples: - audit.log + name: log_file_name type: string log.file.name_resolved: brief: | @@ -1281,6 +1455,7 @@ entries: column: String examples: - uuid.log + name: log_file_name_resolved type: string log.file.path: brief: | @@ -1288,6 +1463,7 @@ entries: column: String examples: - /var/log/mysql/audit.log + name: log_file_path type: string log.file.path_resolved: brief: | @@ -1295,6 +1471,7 @@ entries: column: String examples: - /var/lib/docker/uuid.log + name: log_file_path_resolved type: string log.iostream: brief: | @@ -1303,6 +1480,7 @@ entries: enum: - stdout - stderr + name: log_iostream type: string log.record.uid: brief: | @@ -1310,15 +1488,18 @@ entries: column: String examples: - 01ARZ3NDEKTSV4RRFFQ69G5FAV + name: log_record_uid type: string message.compressed_size: brief: Compressed size of the message in bytes. column: Int64 + name: message_compressed_size type: int message.id: brief: MUST be calculated as two different counters starting from `1` one for sent messages and one for received message. column: Int64 + name: message_id type: int message.type: brief: Whether this is a received or sent message. @@ -1326,10 +1507,12 @@ entries: enum: - SENT - RECEIVED + name: message_type type: string message.uncompressed_size: brief: Uncompressed size of the message in bytes. column: Int64 + name: message_uncompressed_size type: int messaging.batch.message_count: brief: The number of messages sent, received, or processed in the scope of the @@ -1339,6 +1522,7 @@ entries: - 0 - 1 - 2 + name: messaging_batch_message_count type: int messaging.client_id: brief: | @@ -1347,11 +1531,13 @@ entries: examples: - client-5 - myhost@8742@s8083jm + name: messaging_client_id type: string messaging.destination.anonymous: brief: A boolean that is true if the message destination is anonymous (could be unnamed or have auto-generated name). column: Bool + name: messaging_destination_anonymous type: boolean messaging.destination.name: brief: The message destination name @@ -1359,22 +1545,26 @@ entries: examples: - MyQueue - MyTopic + name: messaging_destination_name type: string messaging.destination.template: brief: Low cardinality representation of the messaging destination name column: String examples: - /customers/{customerId} + name: messaging_destination_template type: string messaging.destination.temporary: brief: A boolean that is true if the message destination is temporary and might not exist anymore after messages are processed. column: Bool + name: messaging_destination_temporary type: boolean messaging.destination_publish.anonymous: brief: A boolean that is true if the publish message destination is anonymous (could be unnamed or have auto-generated name). column: Bool + name: messaging_destination_publish_anonymous type: boolean messaging.destination_publish.name: brief: The name of the original destination the message was published to @@ -1382,6 +1572,7 @@ entries: examples: - MyQueue - MyTopic + name: messaging_destination_publish_name type: string messaging.gcp_pubsub.message.ordering_key: brief: | @@ -1389,6 +1580,7 @@ entries: column: String examples: - ordering_key + name: messaging_gcp_pubsub_message_ordering_key type: string messaging.kafka.consumer.group: brief: | @@ -1396,6 +1588,7 @@ entries: column: String examples: - my-group + name: messaging_kafka_consumer_group type: string messaging.kafka.destination.partition: brief: | @@ -1403,6 +1596,7 @@ entries: column: Int64 examples: - 2 + name: messaging_kafka_destination_partition type: int messaging.kafka.message.key: brief: | @@ -1410,6 +1604,7 @@ entries: column: String examples: - myKey + name: messaging_kafka_message_key type: string messaging.kafka.message.offset: brief: | @@ -1417,10 +1612,12 @@ entries: column: Int64 examples: - 42 + name: messaging_kafka_message_offset type: int messaging.kafka.message.tombstone: brief: A boolean that is true if the message is a tombstone. column: Bool + name: messaging_kafka_message_tombstone type: boolean messaging.message.body.size: brief: | @@ -1428,6 +1625,7 @@ entries: column: Int64 examples: - 1439 + name: messaging_message_body_size type: int messaging.message.conversation_id: brief: | @@ -1435,6 +1633,7 @@ entries: column: String examples: - MyConversationId + name: messaging_message_conversation_id type: string messaging.message.envelope.size: brief: | @@ -1442,6 +1641,7 @@ entries: column: Int64 examples: - 2738 + name: messaging_message_envelope_size type: int messaging.message.id: brief: A value used by the messaging system as an identifier for the message, @@ -1449,11 +1649,13 @@ entries: column: String examples: - 452a7c7c7c7048c2f887f61572b18fc2 + name: messaging_message_id type: string messaging.operation: brief: | A string identifying the kind of messaging operation. column: String + name: messaging_operation type: string messaging.rabbitmq.destination.routing_key: brief: | @@ -1461,6 +1663,7 @@ entries: column: String examples: - myKey + name: messaging_rabbitmq_destination_routing_key type: string messaging.rocketmq.client_group: brief: | @@ -1468,6 +1671,7 @@ entries: column: String examples: - myConsumerGroup + name: messaging_rocketmq_client_group type: string messaging.rocketmq.consumption_model: brief: | @@ -1476,6 +1680,7 @@ entries: enum: - clustering - broadcasting + name: messaging_rocketmq_consumption_model type: string messaging.rocketmq.message.delay_time_level: brief: | @@ -1483,6 +1688,7 @@ entries: column: Int64 examples: - 3 + name: messaging_rocketmq_message_delay_time_level type: int messaging.rocketmq.message.delivery_timestamp: brief: | @@ -1490,6 +1696,7 @@ entries: column: Int64 examples: - 1665987217045 + name: messaging_rocketmq_message_delivery_timestamp type: int messaging.rocketmq.message.group: brief: | @@ -1497,6 +1704,7 @@ entries: column: String examples: - myMessageGroup + name: messaging_rocketmq_message_group type: string messaging.rocketmq.message.keys: brief: | @@ -1505,6 +1713,7 @@ entries: examples: - keyA - keyB + name: messaging_rocketmq_message_keys type: string[] messaging.rocketmq.message.tag: brief: | @@ -1512,6 +1721,7 @@ entries: column: String examples: - tagA + name: messaging_rocketmq_message_tag type: string messaging.rocketmq.message.type: brief: | @@ -1522,6 +1732,7 @@ entries: - fifo - delay - transaction + name: messaging_rocketmq_message_type type: string messaging.rocketmq.namespace: brief: | @@ -1529,11 +1740,13 @@ entries: column: String examples: - myNamespace + name: messaging_rocketmq_namespace type: string messaging.system: brief: | An identifier for the messaging system being used. See below for a list of well-known identifiers. column: String + name: messaging_system type: string network.carrier.icc: brief: The ISO 3166-1 alpha-2 2-character country code associated with the mobile @@ -1541,24 +1754,28 @@ entries: column: String examples: - DE + name: network_carrier_icc type: string network.carrier.mcc: brief: The mobile carrier country code. column: String examples: - "310" + name: network_carrier_mcc type: string network.carrier.mnc: brief: The mobile carrier network code. column: String examples: - "001" + name: network_carrier_mnc type: string network.carrier.name: brief: The name of the mobile carrier. column: String examples: - sprint + name: network_carrier_name type: string network.connection.subtype: brief: This describes more details regarding the connection.type. It may be the @@ -1567,12 +1784,14 @@ entries: column: String examples: - LTE + name: network_connection_subtype type: string network.connection.type: brief: The internet connection type. column: String examples: - wifi + name: network_connection_type type: string network.io.direction: brief: The network IO operation direction. @@ -1580,6 +1799,7 @@ entries: enum: - transmit - receive + name: network_io_direction type: string network.local.address: brief: Local address of the network connection - IP address or Unix domain socket @@ -1588,12 +1808,14 @@ entries: examples: - 10.1.2.80 - /tmp/my.sock + name: network_local_address type: string network.local.port: brief: Local port number of the network connection. column: UInt16 examples: - 65123 + name: network_local_port type: int network.peer.address: brief: Peer address of the network connection - IP address or Unix domain socket @@ -1602,12 +1824,14 @@ entries: examples: - 10.1.2.80 - /tmp/my.sock + name: network_peer_address type: string network.peer.port: brief: Peer port number of the network connection. column: UInt16 examples: - 65123 + name: network_peer_port type: int network.protocol.name: brief: '[OSI application layer](https://osi-model.com/application-layer/) or non-OSI @@ -1617,12 +1841,14 @@ entries: - amqp - http - mqtt + name: network_protocol_name type: string network.protocol.version: brief: Version of the protocol specified in `network.protocol.name`. column: String examples: - 3.1.1 + name: network_protocol_version type: string network.transport: brief: | @@ -1631,6 +1857,7 @@ entries: examples: - tcp - udp + name: network_transport type: string network.type: brief: '[OSI network layer](https://osi-model.com/network-layer/) or non-OSI equivalent.' @@ -1638,6 +1865,7 @@ entries: examples: - ipv4 - ipv6 + name: network_type type: string oci.manifest.digest: brief: | @@ -1645,6 +1873,7 @@ entries: column: String examples: - sha256:e4ca62c0d62f3e886e684806dfe9d4e0cda60d54986898173c1083856cfda0f4 + name: oci_manifest_digest type: string opentracing.ref_type: brief: Parent-child Reference type @@ -1652,6 +1881,7 @@ entries: enum: - child_of - follows_from + name: opentracing_ref_type type: string os.build_id: brief: Unique identifier for a particular build or compilation of the operating @@ -1661,6 +1891,7 @@ entries: - TQ3C.230805.001.B2 - "20E247" - "22621" + name: os_build_id type: string os.description: brief: | @@ -1669,6 +1900,7 @@ entries: examples: - Microsoft Windows [Version 10.0.18363.778] - Ubuntu 18.04.1 LTS + name: os_description type: string os.name: brief: Human readable operating system name. @@ -1677,11 +1909,13 @@ entries: - iOS - Android - Ubuntu + name: os_name type: string os.type: brief: | The operating system type. column: String + name: os_type type: string os.version: brief: | @@ -1690,6 +1924,7 @@ entries: examples: - 14.2.1 - 18.04.1 + name: os_version type: string otel.scope.name: brief: The name of the instrumentation scope - (`InstrumentationScope.Name` in @@ -1697,6 +1932,7 @@ entries: column: String examples: - io.opentelemetry.contrib.mongodb + name: otel_scope_name type: string otel.scope.version: brief: The version of the instrumentation scope - (`InstrumentationScope.Version` @@ -1704,6 +1940,7 @@ entries: column: String examples: - 1.0.0 + name: otel_scope_version type: string otel.status_code: brief: Name of the code, either "OK" or "ERROR". MUST NOT be set if the status @@ -1712,12 +1949,14 @@ entries: enum: - OK - ERROR + name: otel_status_code type: string otel.status_description: brief: Description of the Status if it has a value, otherwise not set. column: String examples: - resource not found + name: otel_status_description type: string peer.service: brief: | @@ -1725,6 +1964,7 @@ entries: column: String examples: - AuthTokenCache + name: peer_service type: string pool.name: brief: | @@ -1732,6 +1972,7 @@ entries: column: String examples: - myDataSource + name: pool_name type: string process.command: brief: | @@ -1739,6 +1980,7 @@ entries: column: String examples: - cmd/otelcol + name: process_command type: string process.command_args: brief: | @@ -1747,6 +1989,7 @@ entries: examples: - cmd/otecol - --config=config.yaml + name: process_command_args type: string[] process.command_line: brief: | @@ -1754,6 +1997,7 @@ entries: column: String examples: - C:\cmd\otecol --config="my directory\config.yaml" + name: process_command_line type: string process.executable.name: brief: | @@ -1761,6 +2005,7 @@ entries: column: String examples: - otelcol + name: process_executable_name type: string process.executable.path: brief: | @@ -1768,6 +2013,7 @@ entries: column: String examples: - /usr/bin/cmd/otelcol + name: process_executable_path type: string process.owner: brief: | @@ -1775,6 +2021,7 @@ entries: column: String examples: - root + name: process_owner type: string process.parent_pid: brief: | @@ -1782,6 +2029,7 @@ entries: column: Int64 examples: - 111 + name: process_parent_pid type: int process.pid: brief: | @@ -1789,6 +2037,7 @@ entries: column: Int64 examples: - 1234 + name: process_pid type: int process.runtime.description: brief: | @@ -1796,6 +2045,7 @@ entries: column: String examples: - Eclipse OpenJ9 Eclipse OpenJ9 VM openj9-0.21.0 + name: process_runtime_description type: string process.runtime.name: brief: | @@ -1803,6 +2053,7 @@ entries: column: String examples: - OpenJDK Runtime Environment + name: process_runtime_name type: string process.runtime.version: brief: | @@ -1810,6 +2061,7 @@ entries: column: String examples: - 14.0.2 + name: process_runtime_version type: string rpc.connect_rpc.error_code: brief: The [error codes](https://connect.build/docs/protocol/#error-codes) of @@ -1835,6 +2087,7 @@ entries: - unavailable - data_loss - unauthenticated + name: rpc_connect_rpc_error_code type: string rpc.connect_rpc.request.metadata: brief: | @@ -1842,6 +2095,7 @@ entries: column: Array(String) examples: - rpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"] + name: rpc_connect_rpc_request_metadata type: template[string[]] rpc.connect_rpc.response.metadata: brief: | @@ -1849,6 +2103,7 @@ entries: column: Array(String) examples: - rpc.response.metadata.my-custom-metadata-attribute=["attribute_value"] + name: rpc_connect_rpc_response_metadata type: template[string[]] rpc.grpc.request.metadata: brief: | @@ -1856,6 +2111,7 @@ entries: column: Array(String) examples: - rpc.grpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"] + name: rpc_grpc_request_metadata type: template[string[]] rpc.grpc.response.metadata: brief: | @@ -1863,6 +2119,7 @@ entries: column: Array(String) examples: - rpc.grpc.response.metadata.my-custom-metadata-attribute=["attribute_value"] + name: rpc_grpc_response_metadata type: template[string[]] rpc.grpc.status_code: brief: The [numeric status code](https://github.com/grpc/grpc/blob/v1.33.2/doc/statuscodes.md) @@ -1886,6 +2143,7 @@ entries: - 14 - 15 - 16 + name: rpc_grpc_status_code type: int rpc.jsonrpc.error_code: brief: '`error.code` property of response if it is an error response.' @@ -1893,6 +2151,7 @@ entries: examples: - -32700 - 100 + name: rpc_jsonrpc_error_code type: int rpc.jsonrpc.error_message: brief: '`error.message` property of response if it is an error response.' @@ -1900,6 +2159,7 @@ entries: examples: - Parse error - User already exists + name: rpc_jsonrpc_error_message type: string rpc.jsonrpc.request_id: brief: | @@ -1909,6 +2169,7 @@ entries: - "10" - request-7 - "" + name: rpc_jsonrpc_request_id type: string rpc.jsonrpc.version: brief: Protocol version as in `jsonrpc` property of request/response. Since JSON-RPC @@ -1917,6 +2178,7 @@ entries: examples: - "2.0" - "1.0" + name: rpc_jsonrpc_version type: string rpc.method: brief: The name of the (logical) method being called, must be equal to the $method @@ -1924,6 +2186,7 @@ entries: column: String examples: - exampleMethod + name: rpc_method type: string rpc.service: brief: The full (logical) name of the service being called, including its package @@ -1931,11 +2194,13 @@ entries: column: String examples: - myservice.EchoService + name: rpc_service type: string rpc.system: brief: A string identifying the remoting system. See below for a list of well-known identifiers. column: String + name: rpc_system type: string server.address: brief: Server domain name if available without reverse DNS lookup; otherwise, @@ -1945,6 +2210,7 @@ entries: - example.com - 10.1.2.80 - /tmp/my.sock + name: server_address type: string server.port: brief: Server port number. @@ -1953,6 +2219,7 @@ entries: - 80 - 8080 - 443 + name: server_port type: int service.instance.id: brief: | @@ -1961,6 +2228,7 @@ entries: examples: - my-k8s-pod-deployment-1 - 627cc493-f310-47de-96bd-71410b7dec09 + name: service_instance_id type: string service.name: brief: | @@ -1968,6 +2236,7 @@ entries: column: String examples: - shoppingcart + name: service_name type: string service.namespace: brief: | @@ -1975,6 +2244,7 @@ entries: column: String examples: - Shop + name: service_namespace type: string service.version: brief: | @@ -1983,18 +2253,21 @@ entries: examples: - 2.0.0 - a01dbef8a + name: service_version type: string session.id: brief: A unique id to identify a session. column: String examples: - 00112233-4455-6677-8899-aabbccddeeff + name: session_id type: string session.previous_id: brief: The previous `session.id` for this user, when known. column: String examples: - 00112233-4455-6677-8899-aabbccddeeff + name: session_previous_id type: string source.address: brief: Source address - domain name if available without reverse DNS lookup; otherwise, @@ -2004,6 +2277,7 @@ entries: - source.example.com - 10.1.2.80 - /tmp/my.sock + name: source_address type: string source.port: brief: Source port number @@ -2011,6 +2285,7 @@ entries: examples: - 3389 - 2888 + name: source_port type: int state: brief: The state of a connection in the pool @@ -2018,12 +2293,14 @@ entries: enum: - idle - used + name: state type: string system.cpu.logical_number: brief: The logical CPU number [0..n-1] column: Int64 examples: - 1 + name: system_cpu_logical_number type: int system.cpu.state: brief: The state of the CPU @@ -2031,24 +2308,28 @@ entries: examples: - idle - interrupt + name: system_cpu_state type: string system.device: brief: The device identifier column: String examples: - (identifier) + name: system_device type: string system.filesystem.mode: brief: The filesystem mode column: String examples: - rw, ro + name: system_filesystem_mode type: string system.filesystem.mountpoint: brief: The filesystem mount path column: String examples: - /mnt/data + name: system_filesystem_mountpoint type: string system.filesystem.state: brief: The filesystem state @@ -2057,12 +2338,14 @@ entries: - used - free - reserved + name: system_filesystem_state type: string system.filesystem.type: brief: The filesystem type column: String examples: - ext4 + name: system_filesystem_type type: string system.memory.state: brief: The memory state @@ -2070,6 +2353,7 @@ entries: examples: - free - cached + name: system_memory_state type: string system.network.state: brief: A stateless protocol MUST NOT set this attribute @@ -2089,6 +2373,7 @@ entries: - syn_recv - syn_sent - time_wait + name: system_network_state type: string system.paging.direction: brief: The paging access direction @@ -2096,6 +2381,7 @@ entries: enum: - in - out + name: system_paging_direction type: string system.paging.state: brief: The memory paging state @@ -2103,6 +2389,7 @@ entries: enum: - used - free + name: system_paging_state type: string system.paging.type: brief: The memory paging type @@ -2110,6 +2397,7 @@ entries: enum: - major - minor + name: system_paging_type type: string system.processes.status: brief: | @@ -2117,6 +2405,7 @@ entries: column: String examples: - running + name: system_processes_status type: string telemetry.distro.name: brief: | @@ -2124,6 +2413,7 @@ entries: column: String examples: - parts-unlimited-java + name: telemetry_distro_name type: string telemetry.distro.version: brief: | @@ -2131,11 +2421,13 @@ entries: column: String examples: - 1.2.3 + name: telemetry_distro_version type: string telemetry.sdk.language: brief: | The language of the telemetry SDK. column: String + name: telemetry_sdk_language type: string telemetry.sdk.name: brief: | @@ -2143,6 +2435,7 @@ entries: column: String examples: - opentelemetry + name: telemetry_sdk_name type: string telemetry.sdk.version: brief: | @@ -2150,6 +2443,7 @@ entries: column: String examples: - 1.2.3 + name: telemetry_sdk_version type: string thread.id: brief: | @@ -2157,6 +2451,7 @@ entries: column: Int64 examples: - 42 + name: thread_id type: int thread.name: brief: | @@ -2164,6 +2459,7 @@ entries: column: String examples: - main + name: thread_name type: string tls.cipher: brief: | @@ -2172,6 +2468,7 @@ entries: examples: - TLS_RSA_WITH_3DES_EDE_CBC_SHA - TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256 + name: tls_cipher type: string tls.client.certificate: brief: | @@ -2179,6 +2476,7 @@ entries: column: String examples: - MII... + name: tls_client_certificate type: string tls.client.certificate_chain: brief: | @@ -2187,6 +2485,7 @@ entries: examples: - MII... - MI... + name: tls_client_certificate_chain type: string[] tls.client.hash.md5: brief: | @@ -2194,6 +2493,7 @@ entries: column: String examples: - 0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC + name: tls_client_hash_md5 type: string tls.client.hash.sha1: brief: | @@ -2201,6 +2501,7 @@ entries: column: String examples: - 9E393D93138888D288266C2D915214D1D1CCEB2A + name: tls_client_hash_sha1 type: string tls.client.hash.sha256: brief: | @@ -2208,6 +2509,7 @@ entries: column: String examples: - 0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0 + name: tls_client_hash_sha256 type: string tls.client.issuer: brief: Distinguished name of [subject](https://datatracker.ietf.org/doc/html/rfc5280#section-4.1.2.6) @@ -2215,24 +2517,28 @@ entries: column: String examples: - CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com + name: tls_client_issuer type: string tls.client.ja3: brief: A hash that identifies clients based on how they perform an SSL/TLS handshake. column: String examples: - d4e5b18d6b55c71272893221c96ba240 + name: tls_client_ja3 type: string tls.client.not_after: brief: Date/Time indicating when client certificate is no longer considered valid. column: String examples: - "2021-01-01T00:00:00.000Z" + name: tls_client_not_after type: string tls.client.not_before: brief: Date/Time indicating when client certificate is first considered valid. column: String examples: - "1970-01-01T00:00:00.000Z" + name: tls_client_not_before type: string tls.client.server_name: brief: Also called an SNI, this tells the server which hostname to which the client @@ -2240,6 +2546,7 @@ entries: column: String examples: - opentelemetry.io + name: tls_client_server_name type: string tls.client.subject: brief: Distinguished name of subject of the x.509 certificate presented by the @@ -2247,6 +2554,7 @@ entries: column: String examples: - CN=myclient, OU=Documentation Team, DC=example, DC=com + name: tls_client_subject type: string tls.client.supported_ciphers: brief: Array of ciphers offered by the client during the client hello. @@ -2254,12 +2562,14 @@ entries: examples: - '"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", "..."' + name: tls_client_supported_ciphers type: string[] tls.curve: brief: String indicating the curve used for the given cipher, when applicable column: String examples: - secp256r1 + name: tls_curve type: string tls.established: brief: Boolean flag indicating if the TLS negotiation was successful and transitioned @@ -2267,6 +2577,7 @@ entries: column: Bool examples: - true + name: tls_established type: boolean tls.next_protocol: brief: | @@ -2274,11 +2585,13 @@ entries: column: String examples: - http/1.1 + name: tls_next_protocol type: string tls.protocol.name: brief: | Normalized lowercase protocol name parsed from original string of the negotiated [SSL/TLS protocol version](https://www.openssl.org/docs/man1.1.1/man3/SSL_get_version.html#RETURN-VALUES) column: String + name: tls_protocol_name type: string tls.protocol.version: brief: | @@ -2287,6 +2600,7 @@ entries: examples: - "1.2" - "3" + name: tls_protocol_version type: string tls.resumed: brief: Boolean flag indicating if this TLS connection was resumed from an existing @@ -2294,6 +2608,7 @@ entries: column: Bool examples: - true + name: tls_resumed type: boolean tls.server.certificate: brief: | @@ -2301,6 +2616,7 @@ entries: column: String examples: - MII... + name: tls_server_certificate type: string tls.server.certificate_chain: brief: | @@ -2309,6 +2625,7 @@ entries: examples: - MII... - MI... + name: tls_server_certificate_chain type: string[] tls.server.hash.md5: brief: | @@ -2316,6 +2633,7 @@ entries: column: String examples: - 0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC + name: tls_server_hash_md5 type: string tls.server.hash.sha1: brief: | @@ -2323,6 +2641,7 @@ entries: column: String examples: - 9E393D93138888D288266C2D915214D1D1CCEB2A + name: tls_server_hash_sha1 type: string tls.server.hash.sha256: brief: | @@ -2330,6 +2649,7 @@ entries: column: String examples: - 0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0 + name: tls_server_hash_sha256 type: string tls.server.issuer: brief: Distinguished name of [subject](https://datatracker.ietf.org/doc/html/rfc5280#section-4.1.2.6) @@ -2337,24 +2657,28 @@ entries: column: String examples: - CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com + name: tls_server_issuer type: string tls.server.ja3s: brief: A hash that identifies servers based on how they perform an SSL/TLS handshake. column: String examples: - d4e5b18d6b55c71272893221c96ba240 + name: tls_server_ja3s type: string tls.server.not_after: brief: Date/Time indicating when server certificate is no longer considered valid. column: String examples: - "2021-01-01T00:00:00.000Z" + name: tls_server_not_after type: string tls.server.not_before: brief: Date/Time indicating when server certificate is first considered valid. column: String examples: - "1970-01-01T00:00:00.000Z" + name: tls_server_not_before type: string tls.server.subject: brief: Distinguished name of subject of the x.509 certificate presented by the @@ -2362,6 +2686,7 @@ entries: column: String examples: - CN=myserver, OU=Documentation Team, DC=example, DC=com + name: tls_server_subject type: string url.fragment: brief: The [URI fragment](https://www.rfc-editor.org/rfc/rfc3986#section-3.5) @@ -2369,6 +2694,7 @@ entries: column: String examples: - SemConv + name: url_fragment type: string url.full: brief: Absolute URL describing a network resource according to [RFC3986](https://www.rfc-editor.org/rfc/rfc3986) @@ -2376,18 +2702,21 @@ entries: examples: - https://www.foo.bar/search?q=OpenTelemetry#SemConv - //localhost + name: url_full type: string url.path: brief: The [URI path](https://www.rfc-editor.org/rfc/rfc3986#section-3.3) component column: String examples: - /search + name: url_path type: string url.query: brief: The [URI query](https://www.rfc-editor.org/rfc/rfc3986#section-3.4) component column: String examples: - q=OpenTelemetry + name: url_query type: string url.scheme: brief: The [URI scheme](https://www.rfc-editor.org/rfc/rfc3986#section-3.1) component @@ -2397,6 +2726,7 @@ entries: - https - ftp - telnet + name: url_scheme type: string user_agent.original: brief: | @@ -2406,6 +2736,7 @@ entries: - CERN-LineMode/2.15 libwww/2.17b3 - Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1 + name: user_agent_original type: string webengine.description: brief: | @@ -2413,6 +2744,7 @@ entries: column: String examples: - WildFly Full 21.0.0.Final (WildFly Core 13.0.1.Final) - 2.2.2.Final + name: webengine_description type: string webengine.name: brief: | @@ -2420,6 +2752,7 @@ entries: column: String examples: - WildFly + name: webengine_name type: string webengine.version: brief: | @@ -2427,6 +2760,7 @@ entries: column: String examples: - 21.0.0 + name: webengine_version type: string statistics: deprecated: 24 diff --git a/internal/otelschema/registry_test.go b/internal/otelschema/registry_test.go index 868d3ed0..0f3b9b30 100644 --- a/internal/otelschema/registry_test.go +++ b/internal/otelschema/registry_test.go @@ -22,6 +22,7 @@ type registryEntry struct { Column proto.ColumnType `json:"column"` Examples []any `json:"examples,omitempty"` Brief string `json:"brief,omitempty"` + Name string `json:"name,omitempty"` } func anyTo[T any](s []any) (result []T) { @@ -195,6 +196,7 @@ func TestParseAllAttributes(t *testing.T) { Column: columnType(name, v.Brief.Value, typ, enum), Examples: examples, Brief: v.Brief.Value, + Name: strings.ReplaceAll(name, ".", "_"), } } } From 524536e0b67503346b50bd53f9be706c9a366237 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 15:29:29 +0300 Subject: [PATCH 059/112] feat(otelschema): generate DDL --- internal/otelschema/_golden/ddl.sql | 434 ++++++++++++++++++++++ internal/otelschema/_golden/registry.yaml | 401 ++++++++++++++++++-- internal/otelschema/registry_test.go | 54 ++- 3 files changed, 854 insertions(+), 35 deletions(-) create mode 100644 internal/otelschema/_golden/ddl.sql diff --git a/internal/otelschema/_golden/ddl.sql b/internal/otelschema/_golden/ddl.sql new file mode 100644 index 00000000..ff81dece --- /dev/null +++ b/internal/otelschema/_golden/ddl.sql @@ -0,0 +1,434 @@ +CREATE TABLE columns ( + state Enum8('idle' = 0, 'used' = 1) COMMENT 'state', + + -- android + android_os_api_level String COMMENT 'android.os.api_level', + android_state Enum8('created' = 0, 'background' = 1, 'foreground' = 2) COMMENT 'android.state', + + -- aws + aws_ecs_cluster_arn String COMMENT 'aws.ecs.cluster.arn', + aws_ecs_container_arn String COMMENT 'aws.ecs.container.arn', + aws_ecs_launchtype Enum8('ec2' = 0, 'fargate' = 1) COMMENT 'aws.ecs.launchtype', + aws_ecs_task_arn String COMMENT 'aws.ecs.task.arn', + aws_ecs_task_family String COMMENT 'aws.ecs.task.family', + aws_ecs_task_revision String COMMENT 'aws.ecs.task.revision', + aws_eks_cluster_arn String COMMENT 'aws.eks.cluster.arn', + aws_lambda_invoked_arn String COMMENT 'aws.lambda.invoked_arn', + aws_log_group_arns Array(String) COMMENT 'aws.log.group.arns', + aws_log_group_names Array(String) COMMENT 'aws.log.group.names', + aws_log_stream_arns Array(String) COMMENT 'aws.log.stream.arns', + aws_log_stream_names Array(String) COMMENT 'aws.log.stream.names', + + -- browser + browser_brands Array(String) COMMENT 'browser.brands', + browser_language String COMMENT 'browser.language', + browser_mobile Bool COMMENT 'browser.mobile', + browser_platform String COMMENT 'browser.platform', + + -- client + client_address String COMMENT 'client.address', + client_port UInt16 COMMENT 'client.port', + + -- cloud + cloud_account_id String COMMENT 'cloud.account.id', + cloud_availability_zone String COMMENT 'cloud.availability_zone', + cloud_platform String COMMENT 'cloud.platform', + cloud_provider String COMMENT 'cloud.provider', + cloud_region String COMMENT 'cloud.region', + cloud_resource_id String COMMENT 'cloud.resource_id', + + -- cloudevents + cloudevents_event_id String COMMENT 'cloudevents.event_id', + cloudevents_event_source String COMMENT 'cloudevents.event_source', + cloudevents_event_spec_version String COMMENT 'cloudevents.event_spec_version', + cloudevents_event_subject String COMMENT 'cloudevents.event_subject', + cloudevents_event_type String COMMENT 'cloudevents.event_type', + + -- code + code_column Int64 COMMENT 'code.column', + code_filepath String COMMENT 'code.filepath', + code_function String COMMENT 'code.function', + code_lineno Int64 COMMENT 'code.lineno', + code_namespace String COMMENT 'code.namespace', + code_stacktrace String COMMENT 'code.stacktrace', + + -- container + container_command String COMMENT 'container.command', + container_command_args Array(String) COMMENT 'container.command_args', + container_command_line String COMMENT 'container.command_line', + container_id String COMMENT 'container.id', + container_image_id String COMMENT 'container.image.id', + container_image_name String COMMENT 'container.image.name', + container_image_repo_digests Array(String) COMMENT 'container.image.repo_digests', + container_image_tags Array(String) COMMENT 'container.image.tags', + container_labels String COMMENT 'container.labels', + container_name String COMMENT 'container.name', + container_runtime String COMMENT 'container.runtime', + + -- db + db_cassandra_consistency_level Enum8('all' = 0, 'each_quorum' = 1, 'quorum' = 2, 'local_quorum' = 3, 'one' = 4, 'two' = 5, 'three' = 6, 'local_one' = 7, 'any' = 8, 'serial' = 9, 'local_serial' = 10) COMMENT 'db.cassandra.consistency_level', + db_cassandra_coordinator_dc String COMMENT 'db.cassandra.coordinator.dc', + db_cassandra_coordinator_id String COMMENT 'db.cassandra.coordinator.id', + db_cassandra_idempotence Bool COMMENT 'db.cassandra.idempotence', + db_cassandra_page_size Int64 COMMENT 'db.cassandra.page_size', + db_cassandra_speculative_execution_count Int64 COMMENT 'db.cassandra.speculative_execution_count', + db_cassandra_table String COMMENT 'db.cassandra.table', + db_connection_string String COMMENT 'db.connection_string', + db_cosmosdb_client_id String COMMENT 'db.cosmosdb.client_id', + db_cosmosdb_connection_mode Enum8('gateway' = 0, 'direct' = 1) COMMENT 'db.cosmosdb.connection_mode', + db_cosmosdb_container String COMMENT 'db.cosmosdb.container', + db_cosmosdb_operation_type String COMMENT 'db.cosmosdb.operation_type', + db_cosmosdb_request_charge Float64 COMMENT 'db.cosmosdb.request_charge', + db_cosmosdb_request_content_length Int64 COMMENT 'db.cosmosdb.request_content_length', + db_cosmosdb_status_code Int64 COMMENT 'db.cosmosdb.status_code', + db_cosmosdb_sub_status_code Int64 COMMENT 'db.cosmosdb.sub_status_code', + db_elasticsearch_cluster_name String COMMENT 'db.elasticsearch.cluster.name', + db_elasticsearch_node_name String COMMENT 'db.elasticsearch.node.name', + db_elasticsearch_path_parts String COMMENT 'db.elasticsearch.path_parts', + db_jdbc_driver_classname String COMMENT 'db.jdbc.driver_classname', + db_mongodb_collection String COMMENT 'db.mongodb.collection', + db_mssql_instance_name String COMMENT 'db.mssql.instance_name', + db_name String COMMENT 'db.name', + db_operation String COMMENT 'db.operation', + db_redis_database_index Int64 COMMENT 'db.redis.database_index', + db_sql_table String COMMENT 'db.sql.table', + db_statement String COMMENT 'db.statement', + db_system String COMMENT 'db.system', + db_user String COMMENT 'db.user', + + -- deployment + deployment_environment String COMMENT 'deployment.environment', + + -- destination + destination_address String COMMENT 'destination.address', + destination_port UInt16 COMMENT 'destination.port', + + -- device + device_id String COMMENT 'device.id', + device_manufacturer String COMMENT 'device.manufacturer', + device_model_identifier String COMMENT 'device.model.identifier', + device_model_name String COMMENT 'device.model.name', + + -- disk + disk_io_direction Enum8('read' = 0, 'write' = 1) COMMENT 'disk.io.direction', + + -- enduser + enduser_id String COMMENT 'enduser.id', + enduser_role String COMMENT 'enduser.role', + enduser_scope String COMMENT 'enduser.scope', + + -- error + error_type String COMMENT 'error.type', + + -- event + event_name String COMMENT 'event.name', + + -- exception + exception_escaped Bool COMMENT 'exception.escaped', + exception_message String COMMENT 'exception.message', + exception_stacktrace String COMMENT 'exception.stacktrace', + exception_type String COMMENT 'exception.type', + + -- faas + faas_coldstart Bool COMMENT 'faas.coldstart', + faas_cron String COMMENT 'faas.cron', + faas_document_collection String COMMENT 'faas.document.collection', + faas_document_name String COMMENT 'faas.document.name', + faas_document_operation String COMMENT 'faas.document.operation', + faas_document_time DateTime COMMENT 'faas.document.time', + faas_instance String COMMENT 'faas.instance', + faas_invocation_id String COMMENT 'faas.invocation_id', + faas_invoked_name String COMMENT 'faas.invoked_name', + faas_invoked_provider String COMMENT 'faas.invoked_provider', + faas_invoked_region String COMMENT 'faas.invoked_region', + faas_max_memory Int64 COMMENT 'faas.max_memory', + faas_name String COMMENT 'faas.name', + faas_time DateTime COMMENT 'faas.time', + faas_trigger Enum8('datasource' = 0, 'http' = 1, 'pubsub' = 2, 'timer' = 3, 'other' = 4) COMMENT 'faas.trigger', + faas_version String COMMENT 'faas.version', + + -- feature_flag + feature_flag_key String COMMENT 'feature_flag.key', + feature_flag_provider_name String COMMENT 'feature_flag.provider_name', + feature_flag_variant String COMMENT 'feature_flag.variant', + + -- gcp + gcp_cloud_run_job_execution String COMMENT 'gcp.cloud_run.job.execution', + gcp_cloud_run_job_task_index Int64 COMMENT 'gcp.cloud_run.job.task_index', + gcp_gce_instance_hostname String COMMENT 'gcp.gce.instance.hostname', + gcp_gce_instance_name String COMMENT 'gcp.gce.instance.name', + + -- heroku + heroku_app_id String COMMENT 'heroku.app.id', + heroku_release_commit String COMMENT 'heroku.release.commit', + heroku_release_creation_timestamp String COMMENT 'heroku.release.creation_timestamp', + + -- host + host_arch String COMMENT 'host.arch', + host_cpu_cache_l2_size Int64 COMMENT 'host.cpu.cache.l2.size', + host_cpu_family String COMMENT 'host.cpu.family', + host_cpu_model_id String COMMENT 'host.cpu.model.id', + host_cpu_model_name String COMMENT 'host.cpu.model.name', + host_cpu_stepping Int64 COMMENT 'host.cpu.stepping', + host_cpu_vendor_id String COMMENT 'host.cpu.vendor.id', + host_id String COMMENT 'host.id', + host_image_id String COMMENT 'host.image.id', + host_image_name String COMMENT 'host.image.name', + host_image_version String COMMENT 'host.image.version', + host_ip Array(String) COMMENT 'host.ip', + host_mac Array(String) COMMENT 'host.mac', + host_name String COMMENT 'host.name', + host_type String COMMENT 'host.type', + + -- http + http_request_body_size Int64 COMMENT 'http.request.body.size', + http_request_header Array(String) COMMENT 'http.request.header', + http_request_method String COMMENT 'http.request.method', + http_request_method_original String COMMENT 'http.request.method_original', + http_request_resend_count Int64 COMMENT 'http.request.resend_count', + http_response_body_size Int64 COMMENT 'http.response.body.size', + http_response_header Array(String) COMMENT 'http.response.header', + http_response_status_code Int64 COMMENT 'http.response.status_code', + http_route String COMMENT 'http.route', + + -- ios + ios_state Enum8('active' = 0, 'inactive' = 1, 'background' = 2, 'foreground' = 3, 'terminate' = 4) COMMENT 'ios.state', + + -- jvm + jvm_buffer_pool_name String COMMENT 'jvm.buffer.pool.name', + jvm_gc_action String COMMENT 'jvm.gc.action', + jvm_gc_name String COMMENT 'jvm.gc.name', + jvm_memory_pool_name String COMMENT 'jvm.memory.pool.name', + jvm_memory_type Enum8('heap' = 0, 'non_heap' = 1) COMMENT 'jvm.memory.type', + jvm_thread_daemon Bool COMMENT 'jvm.thread.daemon', + jvm_thread_state Enum8('new' = 0, 'runnable' = 1, 'blocked' = 2, 'waiting' = 3, 'timed_waiting' = 4, 'terminated' = 5) COMMENT 'jvm.thread.state', + + -- k8s + k8s_cluster_name String COMMENT 'k8s.cluster.name', + k8s_cluster_uid UUID COMMENT 'k8s.cluster.uid', + k8s_container_name String COMMENT 'k8s.container.name', + k8s_container_restart_count Int64 COMMENT 'k8s.container.restart_count', + k8s_cronjob_name String COMMENT 'k8s.cronjob.name', + k8s_cronjob_uid UUID COMMENT 'k8s.cronjob.uid', + k8s_daemonset_name String COMMENT 'k8s.daemonset.name', + k8s_daemonset_uid UUID COMMENT 'k8s.daemonset.uid', + k8s_deployment_name String COMMENT 'k8s.deployment.name', + k8s_deployment_uid UUID COMMENT 'k8s.deployment.uid', + k8s_job_name String COMMENT 'k8s.job.name', + k8s_job_uid UUID COMMENT 'k8s.job.uid', + k8s_namespace_name String COMMENT 'k8s.namespace.name', + k8s_node_name String COMMENT 'k8s.node.name', + k8s_node_uid UUID COMMENT 'k8s.node.uid', + k8s_pod_name String COMMENT 'k8s.pod.name', + k8s_pod_uid UUID COMMENT 'k8s.pod.uid', + k8s_replicaset_name String COMMENT 'k8s.replicaset.name', + k8s_replicaset_uid UUID COMMENT 'k8s.replicaset.uid', + k8s_statefulset_name String COMMENT 'k8s.statefulset.name', + k8s_statefulset_uid UUID COMMENT 'k8s.statefulset.uid', + + -- log + log_file_name String COMMENT 'log.file.name', + log_file_name_resolved String COMMENT 'log.file.name_resolved', + log_file_path String COMMENT 'log.file.path', + log_file_path_resolved String COMMENT 'log.file.path_resolved', + log_iostream Enum8('stdout' = 0, 'stderr' = 1) COMMENT 'log.iostream', + log_record_uid String COMMENT 'log.record.uid', + + -- message + message_compressed_size Int64 COMMENT 'message.compressed_size', + message_id Int64 COMMENT 'message.id', + message_type Enum8('SENT' = 0, 'RECEIVED' = 1) COMMENT 'message.type', + message_uncompressed_size Int64 COMMENT 'message.uncompressed_size', + + -- messaging + messaging_batch_message_count Int64 COMMENT 'messaging.batch.message_count', + messaging_client_id String COMMENT 'messaging.client_id', + messaging_destination_anonymous Bool COMMENT 'messaging.destination.anonymous', + messaging_destination_name String COMMENT 'messaging.destination.name', + messaging_destination_publish_anonymous Bool COMMENT 'messaging.destination_publish.anonymous', + messaging_destination_publish_name String COMMENT 'messaging.destination_publish.name', + messaging_destination_template String COMMENT 'messaging.destination.template', + messaging_destination_temporary Bool COMMENT 'messaging.destination.temporary', + messaging_gcp_pubsub_message_ordering_key String COMMENT 'messaging.gcp_pubsub.message.ordering_key', + messaging_kafka_consumer_group String COMMENT 'messaging.kafka.consumer.group', + messaging_kafka_destination_partition Int64 COMMENT 'messaging.kafka.destination.partition', + messaging_kafka_message_key String COMMENT 'messaging.kafka.message.key', + messaging_kafka_message_offset Int64 COMMENT 'messaging.kafka.message.offset', + messaging_kafka_message_tombstone Bool COMMENT 'messaging.kafka.message.tombstone', + messaging_message_body_size Int64 COMMENT 'messaging.message.body.size', + messaging_message_conversation_id String COMMENT 'messaging.message.conversation_id', + messaging_message_envelope_size Int64 COMMENT 'messaging.message.envelope.size', + messaging_message_id String COMMENT 'messaging.message.id', + messaging_operation String COMMENT 'messaging.operation', + messaging_rabbitmq_destination_routing_key String COMMENT 'messaging.rabbitmq.destination.routing_key', + messaging_rocketmq_client_group String COMMENT 'messaging.rocketmq.client_group', + messaging_rocketmq_consumption_model Enum8('clustering' = 0, 'broadcasting' = 1) COMMENT 'messaging.rocketmq.consumption_model', + messaging_rocketmq_message_delay_time_level Int64 COMMENT 'messaging.rocketmq.message.delay_time_level', + messaging_rocketmq_message_delivery_timestamp Int64 COMMENT 'messaging.rocketmq.message.delivery_timestamp', + messaging_rocketmq_message_group String COMMENT 'messaging.rocketmq.message.group', + messaging_rocketmq_message_keys Array(String) COMMENT 'messaging.rocketmq.message.keys', + messaging_rocketmq_message_tag String COMMENT 'messaging.rocketmq.message.tag', + messaging_rocketmq_message_type Enum8('normal' = 0, 'fifo' = 1, 'delay' = 2, 'transaction' = 3) COMMENT 'messaging.rocketmq.message.type', + messaging_rocketmq_namespace String COMMENT 'messaging.rocketmq.namespace', + messaging_system String COMMENT 'messaging.system', + + -- network + network_carrier_icc String COMMENT 'network.carrier.icc', + network_carrier_mcc String COMMENT 'network.carrier.mcc', + network_carrier_mnc String COMMENT 'network.carrier.mnc', + network_carrier_name String COMMENT 'network.carrier.name', + network_connection_subtype String COMMENT 'network.connection.subtype', + network_connection_type String COMMENT 'network.connection.type', + network_io_direction Enum8('transmit' = 0, 'receive' = 1) COMMENT 'network.io.direction', + network_local_address String COMMENT 'network.local.address', + network_local_port UInt16 COMMENT 'network.local.port', + network_peer_address String COMMENT 'network.peer.address', + network_peer_port UInt16 COMMENT 'network.peer.port', + network_protocol_name String COMMENT 'network.protocol.name', + network_protocol_version String COMMENT 'network.protocol.version', + network_transport String COMMENT 'network.transport', + network_type String COMMENT 'network.type', + + -- oci + oci_manifest_digest String COMMENT 'oci.manifest.digest', + + -- opentracing + opentracing_ref_type Enum8('child_of' = 0, 'follows_from' = 1) COMMENT 'opentracing.ref_type', + + -- os + os_build_id String COMMENT 'os.build_id', + os_description String COMMENT 'os.description', + os_name String COMMENT 'os.name', + os_type String COMMENT 'os.type', + os_version String COMMENT 'os.version', + + -- otel + otel_scope_name String COMMENT 'otel.scope.name', + otel_scope_version String COMMENT 'otel.scope.version', + otel_status_code Enum8('OK' = 0, 'ERROR' = 1) COMMENT 'otel.status_code', + otel_status_description String COMMENT 'otel.status_description', + + -- peer + peer_service String COMMENT 'peer.service', + + -- pool + pool_name String COMMENT 'pool.name', + + -- process + process_command String COMMENT 'process.command', + process_command_args Array(String) COMMENT 'process.command_args', + process_command_line String COMMENT 'process.command_line', + process_executable_name String COMMENT 'process.executable.name', + process_executable_path String COMMENT 'process.executable.path', + process_owner String COMMENT 'process.owner', + process_parent_pid Int64 COMMENT 'process.parent_pid', + process_pid Int64 COMMENT 'process.pid', + process_runtime_description String COMMENT 'process.runtime.description', + process_runtime_name String COMMENT 'process.runtime.name', + process_runtime_version String COMMENT 'process.runtime.version', + + -- rpc + rpc_connect_rpc_error_code Enum8('cancelled' = 0, 'unknown' = 1, 'invalid_argument' = 2, 'deadline_exceeded' = 3, 'not_found' = 4, 'already_exists' = 5, 'permission_denied' = 6, 'resource_exhausted' = 7, 'failed_precondition' = 8, 'aborted' = 9, 'out_of_range' = 10, 'unimplemented' = 11, 'internal' = 12, 'unavailable' = 13, 'data_loss' = 14, 'unauthenticated' = 15) COMMENT 'rpc.connect_rpc.error_code', + rpc_connect_rpc_request_metadata Array(String) COMMENT 'rpc.connect_rpc.request.metadata', + rpc_connect_rpc_response_metadata Array(String) COMMENT 'rpc.connect_rpc.response.metadata', + rpc_grpc_request_metadata Array(String) COMMENT 'rpc.grpc.request.metadata', + rpc_grpc_response_metadata Array(String) COMMENT 'rpc.grpc.response.metadata', + rpc_grpc_status_code UInt8 COMMENT 'rpc.grpc.status_code', + rpc_jsonrpc_error_code Int64 COMMENT 'rpc.jsonrpc.error_code', + rpc_jsonrpc_error_message String COMMENT 'rpc.jsonrpc.error_message', + rpc_jsonrpc_request_id String COMMENT 'rpc.jsonrpc.request_id', + rpc_jsonrpc_version String COMMENT 'rpc.jsonrpc.version', + rpc_method String COMMENT 'rpc.method', + rpc_service String COMMENT 'rpc.service', + rpc_system String COMMENT 'rpc.system', + + -- server + server_address String COMMENT 'server.address', + server_port UInt16 COMMENT 'server.port', + + -- service + service_instance_id String COMMENT 'service.instance.id', + service_name String COMMENT 'service.name', + service_namespace String COMMENT 'service.namespace', + service_version String COMMENT 'service.version', + + -- session + session_id String COMMENT 'session.id', + session_previous_id String COMMENT 'session.previous_id', + + -- source + source_address String COMMENT 'source.address', + source_port UInt16 COMMENT 'source.port', + + -- system + system_cpu_logical_number Int64 COMMENT 'system.cpu.logical_number', + system_cpu_state String COMMENT 'system.cpu.state', + system_device String COMMENT 'system.device', + system_filesystem_mode String COMMENT 'system.filesystem.mode', + system_filesystem_mountpoint String COMMENT 'system.filesystem.mountpoint', + system_filesystem_state Enum8('used' = 0, 'free' = 1, 'reserved' = 2) COMMENT 'system.filesystem.state', + system_filesystem_type String COMMENT 'system.filesystem.type', + system_memory_state String COMMENT 'system.memory.state', + system_network_state Enum8('close' = 0, 'close_wait' = 1, 'closing' = 2, 'delete' = 3, 'established' = 4, 'fin_wait_1' = 5, 'fin_wait_2' = 6, 'last_ack' = 7, 'listen' = 8, 'syn_recv' = 9, 'syn_sent' = 10, 'time_wait' = 11) COMMENT 'system.network.state', + system_paging_direction Enum8('in' = 0, 'out' = 1) COMMENT 'system.paging.direction', + system_paging_state Enum8('used' = 0, 'free' = 1) COMMENT 'system.paging.state', + system_paging_type Enum8('major' = 0, 'minor' = 1) COMMENT 'system.paging.type', + system_processes_status String COMMENT 'system.processes.status', + + -- telemetry + telemetry_distro_name String COMMENT 'telemetry.distro.name', + telemetry_distro_version String COMMENT 'telemetry.distro.version', + telemetry_sdk_language String COMMENT 'telemetry.sdk.language', + telemetry_sdk_name String COMMENT 'telemetry.sdk.name', + telemetry_sdk_version String COMMENT 'telemetry.sdk.version', + + -- thread + thread_id Int64 COMMENT 'thread.id', + thread_name String COMMENT 'thread.name', + + -- tls + tls_cipher String COMMENT 'tls.cipher', + tls_client_certificate String COMMENT 'tls.client.certificate', + tls_client_certificate_chain Array(String) COMMENT 'tls.client.certificate_chain', + tls_client_hash_md5 String COMMENT 'tls.client.hash.md5', + tls_client_hash_sha1 String COMMENT 'tls.client.hash.sha1', + tls_client_hash_sha256 String COMMENT 'tls.client.hash.sha256', + tls_client_issuer String COMMENT 'tls.client.issuer', + tls_client_ja3 String COMMENT 'tls.client.ja3', + tls_client_not_after String COMMENT 'tls.client.not_after', + tls_client_not_before String COMMENT 'tls.client.not_before', + tls_client_server_name String COMMENT 'tls.client.server_name', + tls_client_subject String COMMENT 'tls.client.subject', + tls_client_supported_ciphers Array(String) COMMENT 'tls.client.supported_ciphers', + tls_curve String COMMENT 'tls.curve', + tls_established Bool COMMENT 'tls.established', + tls_next_protocol String COMMENT 'tls.next_protocol', + tls_protocol_name String COMMENT 'tls.protocol.name', + tls_protocol_version String COMMENT 'tls.protocol.version', + tls_resumed Bool COMMENT 'tls.resumed', + tls_server_certificate String COMMENT 'tls.server.certificate', + tls_server_certificate_chain Array(String) COMMENT 'tls.server.certificate_chain', + tls_server_hash_md5 String COMMENT 'tls.server.hash.md5', + tls_server_hash_sha1 String COMMENT 'tls.server.hash.sha1', + tls_server_hash_sha256 String COMMENT 'tls.server.hash.sha256', + tls_server_issuer String COMMENT 'tls.server.issuer', + tls_server_ja3s String COMMENT 'tls.server.ja3s', + tls_server_not_after String COMMENT 'tls.server.not_after', + tls_server_not_before String COMMENT 'tls.server.not_before', + tls_server_subject String COMMENT 'tls.server.subject', + + -- url + url_fragment String COMMENT 'url.fragment', + url_full String COMMENT 'url.full', + url_path String COMMENT 'url.path', + url_query String COMMENT 'url.query', + url_scheme String COMMENT 'url.scheme', + + -- user_agent + user_agent_original String COMMENT 'user_agent.original', + + -- webengine + webengine_description String COMMENT 'webengine.description', + webengine_name String COMMENT 'webengine.name', + webengine_version String COMMENT 'webengine.version' +) ENGINE Null; \ No newline at end of file diff --git a/internal/otelschema/_golden/registry.yaml b/internal/otelschema/_golden/registry.yaml index 715b010c..22323127 100644 --- a/internal/otelschema/_golden/registry.yaml +++ b/internal/otelschema/_golden/registry.yaml @@ -6,16 +6,18 @@ entries: examples: - "33" - "32" + group: android name: android_os_api_level type: string android.state: brief: | This attribute represents the state the application has transitioned into at the occurrence of the event. - column: Enum8(0 = 'created', 1 = 'background', 2 = 'foreground') + column: Enum8('created' = 0, 'background' = 1, 'foreground' = 2) enum: - created - background - foreground + group: android name: android_state type: string aws.ecs.cluster.arn: @@ -24,6 +26,7 @@ entries: column: String examples: - arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster + group: aws name: aws_ecs_cluster_arn type: string aws.ecs.container.arn: @@ -32,15 +35,17 @@ entries: column: String examples: - arn:aws:ecs:us-west-1:123456789123:container/32624152-9086-4f0e-acae-1a75b14fe4d9 + group: aws name: aws_ecs_container_arn type: string aws.ecs.launchtype: brief: | The [launch type](https://docs.aws.amazon.com/AmazonECS/latest/developerguide/launch_types.html) for an ECS task. - column: Enum8(0 = 'ec2', 1 = 'fargate') + column: Enum8('ec2' = 0, 'fargate' = 1) enum: - ec2 - fargate + group: aws name: aws_ecs_launchtype type: string aws.ecs.task.arn: @@ -49,6 +54,7 @@ entries: column: String examples: - arn:aws:ecs:us-west-1:123456789123:task/10838bed-421f-43ef-870a-f43feacbbb5b + group: aws name: aws_ecs_task_arn type: string aws.ecs.task.family: @@ -57,6 +63,7 @@ entries: column: String examples: - opentelemetry-family + group: aws name: aws_ecs_task_family type: string aws.ecs.task.revision: @@ -66,6 +73,7 @@ entries: examples: - "8" - "26" + group: aws name: aws_ecs_task_revision type: string aws.eks.cluster.arn: @@ -74,6 +82,7 @@ entries: column: String examples: - arn:aws:ecs:us-west-2:123456789123:cluster/my-cluster + group: aws name: aws_eks_cluster_arn type: string aws.lambda.invoked_arn: @@ -82,6 +91,7 @@ entries: column: String examples: - arn:aws:lambda:us-east-1:123456:function:myfunction:myalias + group: aws name: aws_lambda_invoked_arn type: string aws.log.group.arns: @@ -90,6 +100,7 @@ entries: column: Array(String) examples: - arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:* + group: aws name: aws_log_group_arns type: string[] aws.log.group.names: @@ -99,6 +110,7 @@ entries: examples: - /aws/lambda/my-function - opentelemetry-service + group: aws name: aws_log_group_names type: string[] aws.log.stream.arns: @@ -107,6 +119,7 @@ entries: column: Array(String) examples: - arn:aws:logs:us-west-1:123456789012:log-group:/aws/my/group:log-stream:logs/main/10838bed-421f-43ef-870a-f43feacbbb5b + group: aws name: aws_log_stream_arns type: string[] aws.log.stream.names: @@ -115,6 +128,7 @@ entries: column: Array(String) examples: - logs/main/10838bed-421f-43ef-870a-f43feacbbb5b + group: aws name: aws_log_stream_names type: string[] browser.brands: @@ -124,6 +138,7 @@ entries: - ' Not A;Brand 99' - Chromium 99 - Chrome 99 + group: browser name: browser_brands type: string[] browser.language: @@ -134,11 +149,13 @@ entries: - en-US - fr - fr-FR + group: browser name: browser_language type: string browser.mobile: brief: A boolean that is true if the browser is running on a mobile device column: Bool + group: browser name: browser_mobile type: boolean browser.platform: @@ -148,6 +165,7 @@ entries: - Windows - macOS - Android + group: browser name: browser_platform type: string client.address: @@ -158,6 +176,7 @@ entries: - client.example.com - 10.1.2.80 - /tmp/my.sock + group: client name: client_address type: string client.port: @@ -165,6 +184,7 @@ entries: column: UInt16 examples: - 65123 + group: client name: client_port type: int cloud.account.id: @@ -174,6 +194,7 @@ entries: examples: - "111111111111" - opentelemetry + group: cloud name: cloud_account_id type: string cloud.availability_zone: @@ -182,18 +203,21 @@ entries: column: String examples: - us-east-1c + group: cloud name: cloud_availability_zone type: string cloud.platform: brief: | The cloud platform in use. column: String + group: cloud name: cloud_platform type: string cloud.provider: brief: | Name of the cloud provider. column: String + group: cloud name: cloud_provider type: string cloud.region: @@ -203,6 +227,7 @@ entries: examples: - us-central1 - us-east-1 + group: cloud name: cloud_region type: string cloud.resource_id: @@ -213,6 +238,7 @@ entries: - arn:aws:lambda:REGION:ACCOUNT_ID:function:my-function - //run.googleapis.com/projects/PROJECT_ID/locations/LOCATION_ID/services/SERVICE_ID - /subscriptions//resourceGroups//providers/Microsoft.Web/sites//functions/ + group: cloud name: cloud_resource_id type: string cloudevents.event_id: @@ -222,6 +248,7 @@ entries: examples: - 123e4567-e89b-12d3-a456-426614174000 - "0001" + group: cloudevents name: cloudevents_event_id type: string cloudevents.event_source: @@ -232,6 +259,7 @@ entries: - https://github.com/cloudevents - /cloudevents/spec/pull/123 - my-service + group: cloudevents name: cloudevents_event_source type: string cloudevents.event_spec_version: @@ -240,6 +268,7 @@ entries: column: String examples: - "1.0" + group: cloudevents name: cloudevents_event_spec_version type: string cloudevents.event_subject: @@ -248,6 +277,7 @@ entries: column: String examples: - mynewfile.jpg + group: cloudevents name: cloudevents_event_subject type: string cloudevents.event_type: @@ -257,6 +287,7 @@ entries: examples: - com.github.pull_request.opened - com.example.object.deleted.v2 + group: cloudevents name: cloudevents_event_type type: string code.column: @@ -265,6 +296,7 @@ entries: column: Int64 examples: - 16 + group: code name: code_column type: int code.filepath: @@ -273,6 +305,7 @@ entries: column: String examples: - /usr/local/MyApplication/content_root/app/index.php + group: code name: code_filepath type: string code.function: @@ -281,6 +314,7 @@ entries: column: String examples: - serveRequest + group: code name: code_function type: string code.lineno: @@ -289,6 +323,7 @@ entries: column: Int64 examples: - 42 + group: code name: code_lineno type: int code.namespace: @@ -297,6 +332,7 @@ entries: column: String examples: - com.example.MyHttpService + group: code name: code_namespace type: string code.stacktrace: @@ -306,6 +342,7 @@ entries: examples: - at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n at com.example.GenerateTrace.main(GenerateTrace.java:5) + group: code name: code_stacktrace type: string container.command: @@ -314,6 +351,7 @@ entries: column: String examples: - otelcontribcol + group: container name: container_command type: string container.command_args: @@ -322,6 +360,7 @@ entries: column: Array(String) examples: - otelcontribcol, --config, config.yaml + group: container name: container_command_args type: string[] container.command_line: @@ -330,6 +369,7 @@ entries: column: String examples: - otelcontribcol --config config.yaml + group: container name: container_command_line type: string container.id: @@ -338,6 +378,7 @@ entries: column: String examples: - a3bf90e006b2 + group: container name: container_id type: string container.image.id: @@ -346,6 +387,7 @@ entries: column: String examples: - sha256:19c92d0a00d1b66d897bceaa7319bee0dd38a10a851c60bcec9474aa3f01e50f + group: container name: container_image_id type: string container.image.name: @@ -354,6 +396,7 @@ entries: column: String examples: - gcr.io/opentelemetry/operator + group: container name: container_image_name type: string container.image.repo_digests: @@ -363,6 +406,7 @@ entries: examples: - example@sha256:afcc7f1ac1b49db317a7196c902e61c6c3c4607d63599ee1a82d702d249a0ccb - internal.registry.example.com:5000/example@sha256:b69959407d21e8a062e0416bf13405bb2b71ed7a84dde4158ebafacfa06f5578 + group: container name: container_image_repo_digests type: string[] container.image.tags: @@ -372,6 +416,7 @@ entries: examples: - v1.27.1 - 3.5.7-0 + group: container name: container_image_tags type: string[] container.labels: @@ -380,6 +425,7 @@ entries: column: String examples: - container.labels.app=nginx + group: container name: container_labels type: template[string] container.name: @@ -388,6 +434,7 @@ entries: column: String examples: - opentelemetry-autoconf + group: container name: container_name type: string container.runtime: @@ -398,14 +445,15 @@ entries: - docker - containerd - rkt + group: container name: container_runtime type: string db.cassandra.consistency_level: brief: | The consistency level of the query. Based on consistency values from [CQL](https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/dml/dmlConfigConsistency.html). - column: Enum8(0 = 'all', 1 = 'each_quorum', 2 = 'quorum', 3 = 'local_quorum', - 4 = 'one', 5 = 'two', 6 = 'three', 7 = 'local_one', 8 = 'any', 9 = 'serial', - 10 = 'local_serial') + column: Enum8('all' = 0, 'each_quorum' = 1, 'quorum' = 2, 'local_quorum' = 3, + 'one' = 4, 'two' = 5, 'three' = 6, 'local_one' = 7, 'any' = 8, 'serial' = 9, + 'local_serial' = 10) enum: - all - each_quorum @@ -418,6 +466,7 @@ entries: - any - serial - local_serial + group: db name: db_cassandra_consistency_level type: string db.cassandra.coordinator.dc: @@ -426,6 +475,7 @@ entries: column: String examples: - us-west-2 + group: db name: db_cassandra_coordinator_dc type: string db.cassandra.coordinator.id: @@ -434,12 +484,14 @@ entries: column: String examples: - be13faa2-8574-4d71-926d-27f16cf8a7af + group: db name: db_cassandra_coordinator_id type: string db.cassandra.idempotence: brief: | Whether or not the query is idempotent. column: Bool + group: db name: db_cassandra_idempotence type: boolean db.cassandra.page_size: @@ -448,6 +500,7 @@ entries: column: Int64 examples: - 5000 + group: db name: db_cassandra_page_size type: int db.cassandra.speculative_execution_count: @@ -457,6 +510,7 @@ entries: examples: - 0 - 2 + group: db name: db_cassandra_speculative_execution_count type: int db.cassandra.table: @@ -465,6 +519,7 @@ entries: column: String examples: - mytable + group: db name: db_cassandra_table type: string db.connection_string: @@ -473,6 +528,7 @@ entries: column: String examples: - Server=(localdb)\v11.0;Integrated Security=true; + group: db name: db_connection_string type: string db.cosmosdb.client_id: @@ -480,14 +536,16 @@ entries: column: String examples: - 3ba4827d-4422-483f-b59f-85b74211c11d + group: db name: db_cosmosdb_client_id type: string db.cosmosdb.connection_mode: brief: Cosmos client connection mode. - column: Enum8(0 = 'gateway', 1 = 'direct') + column: Enum8('gateway' = 0, 'direct' = 1) enum: - gateway - direct + group: db name: db_cosmosdb_connection_mode type: string db.cosmosdb.container: @@ -495,11 +553,13 @@ entries: column: String examples: - anystring + group: db name: db_cosmosdb_container type: string db.cosmosdb.operation_type: brief: CosmosDB Operation Type. column: String + group: db name: db_cosmosdb_operation_type type: string db.cosmosdb.request_charge: @@ -508,11 +568,13 @@ entries: examples: - 46.18 - 1 + group: db name: db_cosmosdb_request_charge type: double db.cosmosdb.request_content_length: brief: Request payload size in bytes column: Int64 + group: db name: db_cosmosdb_request_content_length type: int db.cosmosdb.status_code: @@ -521,6 +583,7 @@ entries: examples: - 200 - 201 + group: db name: db_cosmosdb_status_code type: int db.cosmosdb.sub_status_code: @@ -529,6 +592,7 @@ entries: examples: - 1000 - 1002 + group: db name: db_cosmosdb_sub_status_code type: int db.elasticsearch.cluster.name: @@ -537,6 +601,7 @@ entries: column: String examples: - e9106fc68e3044f0b1475b04bf4ffd5f + group: db name: db_elasticsearch_cluster_name type: string db.elasticsearch.node.name: @@ -545,6 +610,7 @@ entries: column: String examples: - instance-0000000001 + group: db name: db_elasticsearch_node_name type: string db.elasticsearch.path_parts: @@ -554,6 +620,7 @@ entries: examples: - db.elasticsearch.path_parts.index=test-index - db.elasticsearch.path_parts.doc_id=123 + group: db name: db_elasticsearch_path_parts type: template[string] db.jdbc.driver_classname: @@ -563,6 +630,7 @@ entries: examples: - org.postgresql.Driver - com.microsoft.sqlserver.jdbc.SQLServerDriver + group: db name: db_jdbc_driver_classname type: string db.mongodb.collection: @@ -572,6 +640,7 @@ entries: examples: - customers - products + group: db name: db_mongodb_collection type: string db.mssql.instance_name: @@ -580,6 +649,7 @@ entries: column: String examples: - MSSQLSERVER + group: db name: db_mssql_instance_name type: string db.name: @@ -589,6 +659,7 @@ entries: examples: - customers - main + group: db name: db_name type: string db.operation: @@ -599,6 +670,7 @@ entries: - findAndModify - HMSET - SELECT + group: db name: db_operation type: string db.redis.database_index: @@ -609,6 +681,7 @@ entries: - 0 - 1 - 15 + group: db name: db_redis_database_index type: int db.sql.table: @@ -618,6 +691,7 @@ entries: examples: - public.users - customers + group: db name: db_sql_table type: string db.statement: @@ -627,12 +701,14 @@ entries: examples: - SELECT * FROM wuser_table - SET mykey "WuValue" + group: db name: db_statement type: string db.system: brief: An identifier for the database management system (DBMS) product being used. See below for a list of well-known identifiers. column: String + group: db name: db_system type: string db.user: @@ -642,6 +718,7 @@ entries: examples: - readonly_user - reporting_user + group: db name: db_user type: string deployment.environment: @@ -651,6 +728,7 @@ entries: examples: - staging - production + group: deployment name: deployment_environment type: string destination.address: @@ -661,6 +739,7 @@ entries: - destination.example.com - 10.1.2.80 - /tmp/my.sock + group: destination name: destination_address type: string destination.port: @@ -669,6 +748,7 @@ entries: examples: - 3389 - 2888 + group: destination name: destination_port type: int device.id: @@ -677,6 +757,7 @@ entries: column: String examples: - 2ab2916d-a51f-4ac8-80ee-45ac31a28092 + group: device name: device_id type: string device.manufacturer: @@ -686,6 +767,7 @@ entries: examples: - Apple - Samsung + group: device name: device_manufacturer type: string device.model.identifier: @@ -695,6 +777,7 @@ entries: examples: - iPhone3,4 - SM-G920F + group: device name: device_model_identifier type: string device.model.name: @@ -704,14 +787,16 @@ entries: examples: - iPhone 6s Plus - Samsung Galaxy S6 + group: device name: device_model_name type: string disk.io.direction: brief: The disk IO operation direction. - column: Enum8(0 = 'read', 1 = 'write') + column: Enum8('read' = 0, 'write' = 1) enum: - read - write + group: disk name: disk_io_direction type: string enduser.id: @@ -720,6 +805,7 @@ entries: column: String examples: - username + group: enduser name: enduser_id type: string enduser.role: @@ -728,6 +814,7 @@ entries: column: String examples: - admin + group: enduser name: enduser_role type: string enduser.scope: @@ -736,6 +823,7 @@ entries: column: String examples: - read:message, write:files + group: enduser name: enduser_scope type: string error.type: @@ -747,6 +835,7 @@ entries: - java.net.UnknownHostException - server_certificate_invalid - "500" + group: error name: error_type type: string event.name: @@ -756,12 +845,14 @@ entries: examples: - browser.mouse.click - device.app.lifecycle + group: event name: event_name type: string exception.escaped: brief: | SHOULD be set to true if the exception event is recorded at a point where it is known that the exception is escaping the scope of the span. column: Bool + group: exception name: exception_escaped type: boolean exception.message: @@ -770,6 +861,7 @@ entries: examples: - Division by zero - Can't convert 'int' object to str implicitly + group: exception name: exception_message type: string exception.stacktrace: @@ -780,6 +872,7 @@ entries: - 'Exception in thread "main" java.lang.RuntimeException: Test exception\n at com.example.GenerateTrace.methodB(GenerateTrace.java:13)\n at com.example.GenerateTrace.methodA(GenerateTrace.java:9)\n at com.example.GenerateTrace.main(GenerateTrace.java:5)' + group: exception name: exception_stacktrace type: string exception.type: @@ -789,12 +882,14 @@ entries: examples: - java.net.ConnectException - OSError + group: exception name: exception_type type: string faas.coldstart: brief: | A boolean that is true if the serverless function is executed for the first time (aka cold-start). column: Bool + group: faas name: faas_coldstart type: boolean faas.cron: @@ -803,6 +898,7 @@ entries: column: String examples: - 0/5 * * * ? * + group: faas name: faas_cron type: string faas.document.collection: @@ -812,6 +908,7 @@ entries: examples: - myBucketName - myDbName + group: faas name: faas_document_collection type: string faas.document.name: @@ -821,11 +918,13 @@ entries: examples: - myFile.txt - myTableName + group: faas name: faas_document_name type: string faas.document.operation: brief: Describes the type of the operation that was performed on the data. column: String + group: faas name: faas_document_operation type: string faas.document.time: @@ -834,6 +933,7 @@ entries: column: DateTime examples: - "2020-01-23T13:47:06Z" + group: faas name: faas_document_time type: string faas.instance: @@ -842,6 +942,7 @@ entries: column: String examples: - 2021/06/28/[$LATEST]2f399eb14537447da05ab2a2e39309de + group: faas name: faas_instance type: string faas.invocation_id: @@ -849,6 +950,7 @@ entries: column: String examples: - af9d5aa4-a685-4c5f-a22b-444f80b3cc28 + group: faas name: faas_invocation_id type: string faas.invoked_name: @@ -857,12 +959,14 @@ entries: column: String examples: - my-function + group: faas name: faas_invoked_name type: string faas.invoked_provider: brief: | The cloud provider of the invoked function. column: String + group: faas name: faas_invoked_provider type: string faas.invoked_region: @@ -871,6 +975,7 @@ entries: column: String examples: - eu-central-1 + group: faas name: faas_invoked_region type: string faas.max_memory: @@ -879,6 +984,7 @@ entries: column: Int64 examples: - 134217728 + group: faas name: faas_max_memory type: int faas.name: @@ -888,6 +994,7 @@ entries: examples: - my-function - myazurefunctionapp/some-function-name + group: faas name: faas_name type: string faas.time: @@ -896,17 +1003,20 @@ entries: column: DateTime examples: - "2020-01-23T13:47:06Z" + group: faas name: faas_time type: string faas.trigger: brief: Type of the trigger which caused this function invocation. - column: Enum8(0 = 'datasource', 1 = 'http', 2 = 'pubsub', 3 = 'timer', 4 = 'other') + column: Enum8('datasource' = 0, 'http' = 1, 'pubsub' = 2, 'timer' = 3, 'other' + = 4) enum: - datasource - http - pubsub - timer - other + group: faas name: faas_trigger type: string faas.version: @@ -915,6 +1025,7 @@ entries: examples: - "26" - pinkfroid-00002 + group: faas name: faas_version type: string feature_flag.key: @@ -922,6 +1033,7 @@ entries: column: String examples: - logo-color + group: feature_flag name: feature_flag_key type: string feature_flag.provider_name: @@ -929,6 +1041,7 @@ entries: column: String examples: - Flag Manager + group: feature_flag name: feature_flag_provider_name type: string feature_flag.variant: @@ -939,6 +1052,7 @@ entries: - red - "true" - "on" + group: feature_flag name: feature_flag_variant type: string gcp.cloud_run.job.execution: @@ -948,6 +1062,7 @@ entries: examples: - job-name-xxxx - sample-job-mdw84 + group: gcp name: gcp_cloud_run_job_execution type: string gcp.cloud_run.job.task_index: @@ -957,6 +1072,7 @@ entries: examples: - 0 - 1 + group: gcp name: gcp_cloud_run_job_task_index type: int gcp.gce.instance.hostname: @@ -966,6 +1082,7 @@ entries: examples: - my-host1234.example.com - sample-vm.us-west1-b.c.my-project.internal + group: gcp name: gcp_gce_instance_hostname type: string gcp.gce.instance.name: @@ -975,6 +1092,7 @@ entries: examples: - instance-1 - my-vm-name + group: gcp name: gcp_gce_instance_name type: string heroku.app.id: @@ -983,6 +1101,7 @@ entries: column: String examples: - 2daa2797-e42b-4624-9322-ec3f968df4da + group: heroku name: heroku_app_id type: string heroku.release.commit: @@ -991,6 +1110,7 @@ entries: column: String examples: - e6134959463efd8966b20e75b913cafe3f5ec + group: heroku name: heroku_release_commit type: string heroku.release.creation_timestamp: @@ -999,12 +1119,14 @@ entries: column: String examples: - "2022-10-23T18:00:42Z" + group: heroku name: heroku_release_creation_timestamp type: string host.arch: brief: | The CPU architecture the host system is running on. column: String + group: host name: host_arch type: string host.cpu.cache.l2.size: @@ -1013,6 +1135,7 @@ entries: column: Int64 examples: - 12288000 + group: host name: host_cpu_cache_l2_size type: int host.cpu.family: @@ -1022,6 +1145,7 @@ entries: examples: - "6" - PA-RISC 1.1e + group: host name: host_cpu_family type: string host.cpu.model.id: @@ -1031,6 +1155,7 @@ entries: examples: - "6" - 9000/778/B180L + group: host name: host_cpu_model_id type: string host.cpu.model.name: @@ -1039,6 +1164,7 @@ entries: column: String examples: - 11th Gen Intel(R) Core(TM) i7-1185G7 @ 3.00GHz + group: host name: host_cpu_model_name type: string host.cpu.stepping: @@ -1047,6 +1173,7 @@ entries: column: Int64 examples: - 1 + group: host name: host_cpu_stepping type: int host.cpu.vendor.id: @@ -1055,6 +1182,7 @@ entries: column: String examples: - GenuineIntel + group: host name: host_cpu_vendor_id type: string host.id: @@ -1063,6 +1191,7 @@ entries: column: String examples: - fdbf79e8af94cb7f9e8df36789187052 + group: host name: host_id type: string host.image.id: @@ -1071,6 +1200,7 @@ entries: column: String examples: - ami-07b06b442921831e5 + group: host name: host_image_id type: string host.image.name: @@ -1080,6 +1210,7 @@ entries: examples: - infra-ami-eks-worker-node-7d4ec78312 - CentOS-8-x86_64-1905 + group: host name: host_image_name type: string host.image.version: @@ -1088,6 +1219,7 @@ entries: column: String examples: - "0.1" + group: host name: host_image_version type: string host.ip: @@ -1097,6 +1229,7 @@ entries: examples: - 192.168.1.140 - fe80::abc2:4a28:737a:609e + group: host name: host_ip type: string[] host.mac: @@ -1106,6 +1239,7 @@ entries: examples: - AC-DE-48-23-45-67 - AC-DE-48-23-45-67-01-9F + group: host name: host_mac type: string[] host.name: @@ -1114,6 +1248,7 @@ entries: column: String examples: - opentelemetry-test + group: host name: host_name type: string host.type: @@ -1122,6 +1257,7 @@ entries: column: String examples: - n1-standard-1 + group: host name: host_type type: string http.request.body.size: @@ -1130,6 +1266,7 @@ entries: column: Int64 examples: - 3495 + group: http name: http_request_body_size type: int http.request.header: @@ -1139,6 +1276,7 @@ entries: examples: - http.request.header.content-type=["application/json"] - http.request.header.x-forwarded-for=["1.2.3.4", "1.2.3.5"] + group: http name: http_request_header type: template[string[]] http.request.method: @@ -1148,6 +1286,7 @@ entries: - GET - POST - HEAD + group: http name: http_request_method type: string http.request.method_original: @@ -1157,6 +1296,7 @@ entries: - GeT - ACL - foo + group: http name: http_request_method_original type: string http.request.resend_count: @@ -1165,6 +1305,7 @@ entries: column: Int64 examples: - 3 + group: http name: http_request_resend_count type: int http.response.body.size: @@ -1173,6 +1314,7 @@ entries: column: Int64 examples: - 3495 + group: http name: http_response_body_size type: int http.response.header: @@ -1182,6 +1324,7 @@ entries: examples: - http.response.header.content-type=["application/json"] - http.response.header.my-custom-header=["abc", "def"] + group: http name: http_response_header type: template[string[]] http.response.status_code: @@ -1189,6 +1332,7 @@ entries: column: Int64 examples: - 200 + group: http name: http_response_status_code type: int http.route: @@ -1198,19 +1342,21 @@ entries: examples: - /users/:userID? - '{controller}/{action}/{id?}' + group: http name: http_route type: string ios.state: brief: | This attribute represents the state the application has transitioned into at the occurrence of the event. - column: Enum8(0 = 'active', 1 = 'inactive', 2 = 'background', 3 = 'foreground', - 4 = 'terminate') + column: Enum8('active' = 0, 'inactive' = 1, 'background' = 2, 'foreground' = 3, + 'terminate' = 4) enum: - active - inactive - background - foreground - terminate + group: ios name: ios_state type: string jvm.buffer.pool.name: @@ -1219,6 +1365,7 @@ entries: examples: - mapped - direct + group: jvm name: jvm_buffer_pool_name type: string jvm.gc.action: @@ -1227,6 +1374,7 @@ entries: examples: - end of minor GC - end of major GC + group: jvm name: jvm_gc_action type: string jvm.gc.name: @@ -1235,6 +1383,7 @@ entries: examples: - G1 Young Generation - G1 Old Generation + group: jvm name: jvm_gc_name type: string jvm.memory.pool.name: @@ -1244,25 +1393,28 @@ entries: - G1 Old Gen - G1 Eden space - G1 Survivor Space + group: jvm name: jvm_memory_pool_name type: string jvm.memory.type: brief: The type of memory. - column: Enum8(0 = 'heap', 1 = 'non_heap') + column: Enum8('heap' = 0, 'non_heap' = 1) enum: - heap - non_heap + group: jvm name: jvm_memory_type type: string jvm.thread.daemon: brief: Whether the thread is daemon or not. column: Bool + group: jvm name: jvm_thread_daemon type: boolean jvm.thread.state: brief: State of the thread. - column: Enum8(0 = 'new', 1 = 'runnable', 2 = 'blocked', 3 = 'waiting', 4 = 'timed_waiting', - 5 = 'terminated') + column: Enum8('new' = 0, 'runnable' = 1, 'blocked' = 2, 'waiting' = 3, 'timed_waiting' + = 4, 'terminated' = 5) enum: - new - runnable @@ -1270,6 +1422,7 @@ entries: - waiting - timed_waiting - terminated + group: jvm name: jvm_thread_state type: string k8s.cluster.name: @@ -1278,6 +1431,7 @@ entries: column: String examples: - opentelemetry-cluster + group: k8s name: k8s_cluster_name type: string k8s.cluster.uid: @@ -1286,6 +1440,7 @@ entries: column: UUID examples: - 218fc5a9-a5f1-4b54-aa05-46717d0ab26d + group: k8s name: k8s_cluster_uid type: string k8s.container.name: @@ -1294,6 +1449,7 @@ entries: column: String examples: - redis + group: k8s name: k8s_container_name type: string k8s.container.restart_count: @@ -1303,6 +1459,7 @@ entries: examples: - 0 - 2 + group: k8s name: k8s_container_restart_count type: int k8s.cronjob.name: @@ -1311,6 +1468,7 @@ entries: column: String examples: - opentelemetry + group: k8s name: k8s_cronjob_name type: string k8s.cronjob.uid: @@ -1319,6 +1477,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + group: k8s name: k8s_cronjob_uid type: string k8s.daemonset.name: @@ -1327,6 +1486,7 @@ entries: column: String examples: - opentelemetry + group: k8s name: k8s_daemonset_name type: string k8s.daemonset.uid: @@ -1335,6 +1495,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + group: k8s name: k8s_daemonset_uid type: string k8s.deployment.name: @@ -1343,6 +1504,7 @@ entries: column: String examples: - opentelemetry + group: k8s name: k8s_deployment_name type: string k8s.deployment.uid: @@ -1351,6 +1513,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + group: k8s name: k8s_deployment_uid type: string k8s.job.name: @@ -1359,6 +1522,7 @@ entries: column: String examples: - opentelemetry + group: k8s name: k8s_job_name type: string k8s.job.uid: @@ -1367,6 +1531,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + group: k8s name: k8s_job_uid type: string k8s.namespace.name: @@ -1375,6 +1540,7 @@ entries: column: String examples: - default + group: k8s name: k8s_namespace_name type: string k8s.node.name: @@ -1383,6 +1549,7 @@ entries: column: String examples: - node-1 + group: k8s name: k8s_node_name type: string k8s.node.uid: @@ -1391,6 +1558,7 @@ entries: column: UUID examples: - 1eb3a0c6-0477-4080-a9cb-0cb7db65c6a2 + group: k8s name: k8s_node_uid type: string k8s.pod.name: @@ -1399,6 +1567,7 @@ entries: column: String examples: - opentelemetry-pod-autoconf + group: k8s name: k8s_pod_name type: string k8s.pod.uid: @@ -1407,6 +1576,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + group: k8s name: k8s_pod_uid type: string k8s.replicaset.name: @@ -1415,6 +1585,7 @@ entries: column: String examples: - opentelemetry + group: k8s name: k8s_replicaset_name type: string k8s.replicaset.uid: @@ -1423,6 +1594,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + group: k8s name: k8s_replicaset_uid type: string k8s.statefulset.name: @@ -1431,6 +1603,7 @@ entries: column: String examples: - opentelemetry + group: k8s name: k8s_statefulset_name type: string k8s.statefulset.uid: @@ -1439,6 +1612,7 @@ entries: column: UUID examples: - 275ecb36-5aa8-4c2a-9c47-d8bb681b9aff + group: k8s name: k8s_statefulset_uid type: string log.file.name: @@ -1447,6 +1621,7 @@ entries: column: String examples: - audit.log + group: log name: log_file_name type: string log.file.name_resolved: @@ -1455,6 +1630,7 @@ entries: column: String examples: - uuid.log + group: log name: log_file_name_resolved type: string log.file.path: @@ -1463,6 +1639,7 @@ entries: column: String examples: - /var/log/mysql/audit.log + group: log name: log_file_path type: string log.file.path_resolved: @@ -1471,15 +1648,17 @@ entries: column: String examples: - /var/lib/docker/uuid.log + group: log name: log_file_path_resolved type: string log.iostream: brief: | The stream associated with the log. See below for a list of well-known values. - column: Enum8(0 = 'stdout', 1 = 'stderr') + column: Enum8('stdout' = 0, 'stderr' = 1) enum: - stdout - stderr + group: log name: log_iostream type: string log.record.uid: @@ -1488,30 +1667,35 @@ entries: column: String examples: - 01ARZ3NDEKTSV4RRFFQ69G5FAV + group: log name: log_record_uid type: string message.compressed_size: brief: Compressed size of the message in bytes. column: Int64 + group: message name: message_compressed_size type: int message.id: brief: MUST be calculated as two different counters starting from `1` one for sent messages and one for received message. column: Int64 + group: message name: message_id type: int message.type: brief: Whether this is a received or sent message. - column: Enum8(0 = 'SENT', 1 = 'RECEIVED') + column: Enum8('SENT' = 0, 'RECEIVED' = 1) enum: - SENT - RECEIVED + group: message name: message_type type: string message.uncompressed_size: brief: Uncompressed size of the message in bytes. column: Int64 + group: message name: message_uncompressed_size type: int messaging.batch.message_count: @@ -1522,6 +1706,7 @@ entries: - 0 - 1 - 2 + group: messaging name: messaging_batch_message_count type: int messaging.client_id: @@ -1531,12 +1716,14 @@ entries: examples: - client-5 - myhost@8742@s8083jm + group: messaging name: messaging_client_id type: string messaging.destination.anonymous: brief: A boolean that is true if the message destination is anonymous (could be unnamed or have auto-generated name). column: Bool + group: messaging name: messaging_destination_anonymous type: boolean messaging.destination.name: @@ -1545,6 +1732,7 @@ entries: examples: - MyQueue - MyTopic + group: messaging name: messaging_destination_name type: string messaging.destination.template: @@ -1552,18 +1740,21 @@ entries: column: String examples: - /customers/{customerId} + group: messaging name: messaging_destination_template type: string messaging.destination.temporary: brief: A boolean that is true if the message destination is temporary and might not exist anymore after messages are processed. column: Bool + group: messaging name: messaging_destination_temporary type: boolean messaging.destination_publish.anonymous: brief: A boolean that is true if the publish message destination is anonymous (could be unnamed or have auto-generated name). column: Bool + group: messaging name: messaging_destination_publish_anonymous type: boolean messaging.destination_publish.name: @@ -1572,6 +1763,7 @@ entries: examples: - MyQueue - MyTopic + group: messaging name: messaging_destination_publish_name type: string messaging.gcp_pubsub.message.ordering_key: @@ -1580,6 +1772,7 @@ entries: column: String examples: - ordering_key + group: messaging name: messaging_gcp_pubsub_message_ordering_key type: string messaging.kafka.consumer.group: @@ -1588,6 +1781,7 @@ entries: column: String examples: - my-group + group: messaging name: messaging_kafka_consumer_group type: string messaging.kafka.destination.partition: @@ -1596,6 +1790,7 @@ entries: column: Int64 examples: - 2 + group: messaging name: messaging_kafka_destination_partition type: int messaging.kafka.message.key: @@ -1604,6 +1799,7 @@ entries: column: String examples: - myKey + group: messaging name: messaging_kafka_message_key type: string messaging.kafka.message.offset: @@ -1612,11 +1808,13 @@ entries: column: Int64 examples: - 42 + group: messaging name: messaging_kafka_message_offset type: int messaging.kafka.message.tombstone: brief: A boolean that is true if the message is a tombstone. column: Bool + group: messaging name: messaging_kafka_message_tombstone type: boolean messaging.message.body.size: @@ -1625,6 +1823,7 @@ entries: column: Int64 examples: - 1439 + group: messaging name: messaging_message_body_size type: int messaging.message.conversation_id: @@ -1633,6 +1832,7 @@ entries: column: String examples: - MyConversationId + group: messaging name: messaging_message_conversation_id type: string messaging.message.envelope.size: @@ -1641,6 +1841,7 @@ entries: column: Int64 examples: - 2738 + group: messaging name: messaging_message_envelope_size type: int messaging.message.id: @@ -1649,12 +1850,14 @@ entries: column: String examples: - 452a7c7c7c7048c2f887f61572b18fc2 + group: messaging name: messaging_message_id type: string messaging.operation: brief: | A string identifying the kind of messaging operation. column: String + group: messaging name: messaging_operation type: string messaging.rabbitmq.destination.routing_key: @@ -1663,6 +1866,7 @@ entries: column: String examples: - myKey + group: messaging name: messaging_rabbitmq_destination_routing_key type: string messaging.rocketmq.client_group: @@ -1671,15 +1875,17 @@ entries: column: String examples: - myConsumerGroup + group: messaging name: messaging_rocketmq_client_group type: string messaging.rocketmq.consumption_model: brief: | Model of message consumption. This only applies to consumer spans. - column: Enum8(0 = 'clustering', 1 = 'broadcasting') + column: Enum8('clustering' = 0, 'broadcasting' = 1) enum: - clustering - broadcasting + group: messaging name: messaging_rocketmq_consumption_model type: string messaging.rocketmq.message.delay_time_level: @@ -1688,6 +1894,7 @@ entries: column: Int64 examples: - 3 + group: messaging name: messaging_rocketmq_message_delay_time_level type: int messaging.rocketmq.message.delivery_timestamp: @@ -1696,6 +1903,7 @@ entries: column: Int64 examples: - 1665987217045 + group: messaging name: messaging_rocketmq_message_delivery_timestamp type: int messaging.rocketmq.message.group: @@ -1704,6 +1912,7 @@ entries: column: String examples: - myMessageGroup + group: messaging name: messaging_rocketmq_message_group type: string messaging.rocketmq.message.keys: @@ -1713,6 +1922,7 @@ entries: examples: - keyA - keyB + group: messaging name: messaging_rocketmq_message_keys type: string[] messaging.rocketmq.message.tag: @@ -1721,17 +1931,19 @@ entries: column: String examples: - tagA + group: messaging name: messaging_rocketmq_message_tag type: string messaging.rocketmq.message.type: brief: | Type of message. - column: Enum8(0 = 'normal', 1 = 'fifo', 2 = 'delay', 3 = 'transaction') + column: Enum8('normal' = 0, 'fifo' = 1, 'delay' = 2, 'transaction' = 3) enum: - normal - fifo - delay - transaction + group: messaging name: messaging_rocketmq_message_type type: string messaging.rocketmq.namespace: @@ -1740,12 +1952,14 @@ entries: column: String examples: - myNamespace + group: messaging name: messaging_rocketmq_namespace type: string messaging.system: brief: | An identifier for the messaging system being used. See below for a list of well-known identifiers. column: String + group: messaging name: messaging_system type: string network.carrier.icc: @@ -1754,6 +1968,7 @@ entries: column: String examples: - DE + group: network name: network_carrier_icc type: string network.carrier.mcc: @@ -1761,6 +1976,7 @@ entries: column: String examples: - "310" + group: network name: network_carrier_mcc type: string network.carrier.mnc: @@ -1768,6 +1984,7 @@ entries: column: String examples: - "001" + group: network name: network_carrier_mnc type: string network.carrier.name: @@ -1775,6 +1992,7 @@ entries: column: String examples: - sprint + group: network name: network_carrier_name type: string network.connection.subtype: @@ -1784,6 +2002,7 @@ entries: column: String examples: - LTE + group: network name: network_connection_subtype type: string network.connection.type: @@ -1791,14 +2010,16 @@ entries: column: String examples: - wifi + group: network name: network_connection_type type: string network.io.direction: brief: The network IO operation direction. - column: Enum8(0 = 'transmit', 1 = 'receive') + column: Enum8('transmit' = 0, 'receive' = 1) enum: - transmit - receive + group: network name: network_io_direction type: string network.local.address: @@ -1808,6 +2029,7 @@ entries: examples: - 10.1.2.80 - /tmp/my.sock + group: network name: network_local_address type: string network.local.port: @@ -1815,6 +2037,7 @@ entries: column: UInt16 examples: - 65123 + group: network name: network_local_port type: int network.peer.address: @@ -1824,6 +2047,7 @@ entries: examples: - 10.1.2.80 - /tmp/my.sock + group: network name: network_peer_address type: string network.peer.port: @@ -1831,6 +2055,7 @@ entries: column: UInt16 examples: - 65123 + group: network name: network_peer_port type: int network.protocol.name: @@ -1841,6 +2066,7 @@ entries: - amqp - http - mqtt + group: network name: network_protocol_name type: string network.protocol.version: @@ -1848,6 +2074,7 @@ entries: column: String examples: - 3.1.1 + group: network name: network_protocol_version type: string network.transport: @@ -1857,6 +2084,7 @@ entries: examples: - tcp - udp + group: network name: network_transport type: string network.type: @@ -1865,6 +2093,7 @@ entries: examples: - ipv4 - ipv6 + group: network name: network_type type: string oci.manifest.digest: @@ -1873,14 +2102,16 @@ entries: column: String examples: - sha256:e4ca62c0d62f3e886e684806dfe9d4e0cda60d54986898173c1083856cfda0f4 + group: oci name: oci_manifest_digest type: string opentracing.ref_type: brief: Parent-child Reference type - column: Enum8(0 = 'child_of', 1 = 'follows_from') + column: Enum8('child_of' = 0, 'follows_from' = 1) enum: - child_of - follows_from + group: opentracing name: opentracing_ref_type type: string os.build_id: @@ -1891,6 +2122,7 @@ entries: - TQ3C.230805.001.B2 - "20E247" - "22621" + group: os name: os_build_id type: string os.description: @@ -1900,6 +2132,7 @@ entries: examples: - Microsoft Windows [Version 10.0.18363.778] - Ubuntu 18.04.1 LTS + group: os name: os_description type: string os.name: @@ -1909,12 +2142,14 @@ entries: - iOS - Android - Ubuntu + group: os name: os_name type: string os.type: brief: | The operating system type. column: String + group: os name: os_type type: string os.version: @@ -1924,6 +2159,7 @@ entries: examples: - 14.2.1 - 18.04.1 + group: os name: os_version type: string otel.scope.name: @@ -1932,6 +2168,7 @@ entries: column: String examples: - io.opentelemetry.contrib.mongodb + group: otel name: otel_scope_name type: string otel.scope.version: @@ -1940,15 +2177,17 @@ entries: column: String examples: - 1.0.0 + group: otel name: otel_scope_version type: string otel.status_code: brief: Name of the code, either "OK" or "ERROR". MUST NOT be set if the status code is UNSET. - column: Enum8(0 = 'OK', 1 = 'ERROR') + column: Enum8('OK' = 0, 'ERROR' = 1) enum: - OK - ERROR + group: otel name: otel_status_code type: string otel.status_description: @@ -1956,6 +2195,7 @@ entries: column: String examples: - resource not found + group: otel name: otel_status_description type: string peer.service: @@ -1964,6 +2204,7 @@ entries: column: String examples: - AuthTokenCache + group: peer name: peer_service type: string pool.name: @@ -1972,6 +2213,7 @@ entries: column: String examples: - myDataSource + group: pool name: pool_name type: string process.command: @@ -1980,6 +2222,7 @@ entries: column: String examples: - cmd/otelcol + group: process name: process_command type: string process.command_args: @@ -1989,6 +2232,7 @@ entries: examples: - cmd/otecol - --config=config.yaml + group: process name: process_command_args type: string[] process.command_line: @@ -1997,6 +2241,7 @@ entries: column: String examples: - C:\cmd\otecol --config="my directory\config.yaml" + group: process name: process_command_line type: string process.executable.name: @@ -2005,6 +2250,7 @@ entries: column: String examples: - otelcol + group: process name: process_executable_name type: string process.executable.path: @@ -2013,6 +2259,7 @@ entries: column: String examples: - /usr/bin/cmd/otelcol + group: process name: process_executable_path type: string process.owner: @@ -2021,6 +2268,7 @@ entries: column: String examples: - root + group: process name: process_owner type: string process.parent_pid: @@ -2029,6 +2277,7 @@ entries: column: Int64 examples: - 111 + group: process name: process_parent_pid type: int process.pid: @@ -2037,6 +2286,7 @@ entries: column: Int64 examples: - 1234 + group: process name: process_pid type: int process.runtime.description: @@ -2045,6 +2295,7 @@ entries: column: String examples: - Eclipse OpenJ9 Eclipse OpenJ9 VM openj9-0.21.0 + group: process name: process_runtime_description type: string process.runtime.name: @@ -2053,6 +2304,7 @@ entries: column: String examples: - OpenJDK Runtime Environment + group: process name: process_runtime_name type: string process.runtime.version: @@ -2061,15 +2313,17 @@ entries: column: String examples: - 14.0.2 + group: process name: process_runtime_version type: string rpc.connect_rpc.error_code: brief: The [error codes](https://connect.build/docs/protocol/#error-codes) of the Connect request. Error codes are always string values. - column: Enum8(0 = 'cancelled', 1 = 'unknown', 2 = 'invalid_argument', 3 = 'deadline_exceeded', - 4 = 'not_found', 5 = 'already_exists', 6 = 'permission_denied', 7 = 'resource_exhausted', - 8 = 'failed_precondition', 9 = 'aborted', 10 = 'out_of_range', 11 = 'unimplemented', - 12 = 'internal', 13 = 'unavailable', 14 = 'data_loss', 15 = 'unauthenticated') + column: Enum8('cancelled' = 0, 'unknown' = 1, 'invalid_argument' = 2, 'deadline_exceeded' + = 3, 'not_found' = 4, 'already_exists' = 5, 'permission_denied' = 6, 'resource_exhausted' + = 7, 'failed_precondition' = 8, 'aborted' = 9, 'out_of_range' = 10, 'unimplemented' + = 11, 'internal' = 12, 'unavailable' = 13, 'data_loss' = 14, 'unauthenticated' + = 15) enum: - cancelled - unknown @@ -2087,6 +2341,7 @@ entries: - unavailable - data_loss - unauthenticated + group: rpc name: rpc_connect_rpc_error_code type: string rpc.connect_rpc.request.metadata: @@ -2095,6 +2350,7 @@ entries: column: Array(String) examples: - rpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"] + group: rpc name: rpc_connect_rpc_request_metadata type: template[string[]] rpc.connect_rpc.response.metadata: @@ -2103,6 +2359,7 @@ entries: column: Array(String) examples: - rpc.response.metadata.my-custom-metadata-attribute=["attribute_value"] + group: rpc name: rpc_connect_rpc_response_metadata type: template[string[]] rpc.grpc.request.metadata: @@ -2111,6 +2368,7 @@ entries: column: Array(String) examples: - rpc.grpc.request.metadata.my-custom-metadata-attribute=["1.2.3.4", "1.2.3.5"] + group: rpc name: rpc_grpc_request_metadata type: template[string[]] rpc.grpc.response.metadata: @@ -2119,6 +2377,7 @@ entries: column: Array(String) examples: - rpc.grpc.response.metadata.my-custom-metadata-attribute=["attribute_value"] + group: rpc name: rpc_grpc_response_metadata type: template[string[]] rpc.grpc.status_code: @@ -2143,6 +2402,7 @@ entries: - 14 - 15 - 16 + group: rpc name: rpc_grpc_status_code type: int rpc.jsonrpc.error_code: @@ -2151,6 +2411,7 @@ entries: examples: - -32700 - 100 + group: rpc name: rpc_jsonrpc_error_code type: int rpc.jsonrpc.error_message: @@ -2159,6 +2420,7 @@ entries: examples: - Parse error - User already exists + group: rpc name: rpc_jsonrpc_error_message type: string rpc.jsonrpc.request_id: @@ -2169,6 +2431,7 @@ entries: - "10" - request-7 - "" + group: rpc name: rpc_jsonrpc_request_id type: string rpc.jsonrpc.version: @@ -2178,6 +2441,7 @@ entries: examples: - "2.0" - "1.0" + group: rpc name: rpc_jsonrpc_version type: string rpc.method: @@ -2186,6 +2450,7 @@ entries: column: String examples: - exampleMethod + group: rpc name: rpc_method type: string rpc.service: @@ -2194,12 +2459,14 @@ entries: column: String examples: - myservice.EchoService + group: rpc name: rpc_service type: string rpc.system: brief: A string identifying the remoting system. See below for a list of well-known identifiers. column: String + group: rpc name: rpc_system type: string server.address: @@ -2210,6 +2477,7 @@ entries: - example.com - 10.1.2.80 - /tmp/my.sock + group: server name: server_address type: string server.port: @@ -2219,6 +2487,7 @@ entries: - 80 - 8080 - 443 + group: server name: server_port type: int service.instance.id: @@ -2228,6 +2497,7 @@ entries: examples: - my-k8s-pod-deployment-1 - 627cc493-f310-47de-96bd-71410b7dec09 + group: service name: service_instance_id type: string service.name: @@ -2236,6 +2506,7 @@ entries: column: String examples: - shoppingcart + group: service name: service_name type: string service.namespace: @@ -2244,6 +2515,7 @@ entries: column: String examples: - Shop + group: service name: service_namespace type: string service.version: @@ -2253,6 +2525,7 @@ entries: examples: - 2.0.0 - a01dbef8a + group: service name: service_version type: string session.id: @@ -2260,6 +2533,7 @@ entries: column: String examples: - 00112233-4455-6677-8899-aabbccddeeff + group: session name: session_id type: string session.previous_id: @@ -2267,6 +2541,7 @@ entries: column: String examples: - 00112233-4455-6677-8899-aabbccddeeff + group: session name: session_previous_id type: string source.address: @@ -2277,6 +2552,7 @@ entries: - source.example.com - 10.1.2.80 - /tmp/my.sock + group: source name: source_address type: string source.port: @@ -2285,11 +2561,12 @@ entries: examples: - 3389 - 2888 + group: source name: source_port type: int state: brief: The state of a connection in the pool - column: Enum8(0 = 'idle', 1 = 'used') + column: Enum8('idle' = 0, 'used' = 1) enum: - idle - used @@ -2300,6 +2577,7 @@ entries: column: Int64 examples: - 1 + group: system name: system_cpu_logical_number type: int system.cpu.state: @@ -2308,6 +2586,7 @@ entries: examples: - idle - interrupt + group: system name: system_cpu_state type: string system.device: @@ -2315,6 +2594,7 @@ entries: column: String examples: - (identifier) + group: system name: system_device type: string system.filesystem.mode: @@ -2322,6 +2602,7 @@ entries: column: String examples: - rw, ro + group: system name: system_filesystem_mode type: string system.filesystem.mountpoint: @@ -2329,15 +2610,17 @@ entries: column: String examples: - /mnt/data + group: system name: system_filesystem_mountpoint type: string system.filesystem.state: brief: The filesystem state - column: Enum8(0 = 'used', 1 = 'free', 2 = 'reserved') + column: Enum8('used' = 0, 'free' = 1, 'reserved' = 2) enum: - used - free - reserved + group: system name: system_filesystem_state type: string system.filesystem.type: @@ -2345,6 +2628,7 @@ entries: column: String examples: - ext4 + group: system name: system_filesystem_type type: string system.memory.state: @@ -2353,13 +2637,14 @@ entries: examples: - free - cached + group: system name: system_memory_state type: string system.network.state: brief: A stateless protocol MUST NOT set this attribute - column: Enum8(0 = 'close', 1 = 'close_wait', 2 = 'closing', 3 = 'delete', 4 = - 'established', 5 = 'fin_wait_1', 6 = 'fin_wait_2', 7 = 'last_ack', 8 = 'listen', - 9 = 'syn_recv', 10 = 'syn_sent', 11 = 'time_wait') + column: Enum8('close' = 0, 'close_wait' = 1, 'closing' = 2, 'delete' = 3, 'established' + = 4, 'fin_wait_1' = 5, 'fin_wait_2' = 6, 'last_ack' = 7, 'listen' = 8, 'syn_recv' + = 9, 'syn_sent' = 10, 'time_wait' = 11) enum: - close - close_wait @@ -2373,30 +2658,34 @@ entries: - syn_recv - syn_sent - time_wait + group: system name: system_network_state type: string system.paging.direction: brief: The paging access direction - column: Enum8(0 = 'in', 1 = 'out') + column: Enum8('in' = 0, 'out' = 1) enum: - in - out + group: system name: system_paging_direction type: string system.paging.state: brief: The memory paging state - column: Enum8(0 = 'used', 1 = 'free') + column: Enum8('used' = 0, 'free' = 1) enum: - used - free + group: system name: system_paging_state type: string system.paging.type: brief: The memory paging type - column: Enum8(0 = 'major', 1 = 'minor') + column: Enum8('major' = 0, 'minor' = 1) enum: - major - minor + group: system name: system_paging_type type: string system.processes.status: @@ -2405,6 +2694,7 @@ entries: column: String examples: - running + group: system name: system_processes_status type: string telemetry.distro.name: @@ -2413,6 +2703,7 @@ entries: column: String examples: - parts-unlimited-java + group: telemetry name: telemetry_distro_name type: string telemetry.distro.version: @@ -2421,12 +2712,14 @@ entries: column: String examples: - 1.2.3 + group: telemetry name: telemetry_distro_version type: string telemetry.sdk.language: brief: | The language of the telemetry SDK. column: String + group: telemetry name: telemetry_sdk_language type: string telemetry.sdk.name: @@ -2435,6 +2728,7 @@ entries: column: String examples: - opentelemetry + group: telemetry name: telemetry_sdk_name type: string telemetry.sdk.version: @@ -2443,6 +2737,7 @@ entries: column: String examples: - 1.2.3 + group: telemetry name: telemetry_sdk_version type: string thread.id: @@ -2451,6 +2746,7 @@ entries: column: Int64 examples: - 42 + group: thread name: thread_id type: int thread.name: @@ -2459,6 +2755,7 @@ entries: column: String examples: - main + group: thread name: thread_name type: string tls.cipher: @@ -2468,6 +2765,7 @@ entries: examples: - TLS_RSA_WITH_3DES_EDE_CBC_SHA - TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256 + group: tls name: tls_cipher type: string tls.client.certificate: @@ -2476,6 +2774,7 @@ entries: column: String examples: - MII... + group: tls name: tls_client_certificate type: string tls.client.certificate_chain: @@ -2485,6 +2784,7 @@ entries: examples: - MII... - MI... + group: tls name: tls_client_certificate_chain type: string[] tls.client.hash.md5: @@ -2493,6 +2793,7 @@ entries: column: String examples: - 0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC + group: tls name: tls_client_hash_md5 type: string tls.client.hash.sha1: @@ -2501,6 +2802,7 @@ entries: column: String examples: - 9E393D93138888D288266C2D915214D1D1CCEB2A + group: tls name: tls_client_hash_sha1 type: string tls.client.hash.sha256: @@ -2509,6 +2811,7 @@ entries: column: String examples: - 0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0 + group: tls name: tls_client_hash_sha256 type: string tls.client.issuer: @@ -2517,6 +2820,7 @@ entries: column: String examples: - CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com + group: tls name: tls_client_issuer type: string tls.client.ja3: @@ -2524,6 +2828,7 @@ entries: column: String examples: - d4e5b18d6b55c71272893221c96ba240 + group: tls name: tls_client_ja3 type: string tls.client.not_after: @@ -2531,6 +2836,7 @@ entries: column: String examples: - "2021-01-01T00:00:00.000Z" + group: tls name: tls_client_not_after type: string tls.client.not_before: @@ -2538,6 +2844,7 @@ entries: column: String examples: - "1970-01-01T00:00:00.000Z" + group: tls name: tls_client_not_before type: string tls.client.server_name: @@ -2546,6 +2853,7 @@ entries: column: String examples: - opentelemetry.io + group: tls name: tls_client_server_name type: string tls.client.subject: @@ -2554,6 +2862,7 @@ entries: column: String examples: - CN=myclient, OU=Documentation Team, DC=example, DC=com + group: tls name: tls_client_subject type: string tls.client.supported_ciphers: @@ -2562,6 +2871,7 @@ entries: examples: - '"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", "..."' + group: tls name: tls_client_supported_ciphers type: string[] tls.curve: @@ -2569,6 +2879,7 @@ entries: column: String examples: - secp256r1 + group: tls name: tls_curve type: string tls.established: @@ -2577,6 +2888,7 @@ entries: column: Bool examples: - true + group: tls name: tls_established type: boolean tls.next_protocol: @@ -2585,12 +2897,14 @@ entries: column: String examples: - http/1.1 + group: tls name: tls_next_protocol type: string tls.protocol.name: brief: | Normalized lowercase protocol name parsed from original string of the negotiated [SSL/TLS protocol version](https://www.openssl.org/docs/man1.1.1/man3/SSL_get_version.html#RETURN-VALUES) column: String + group: tls name: tls_protocol_name type: string tls.protocol.version: @@ -2600,6 +2914,7 @@ entries: examples: - "1.2" - "3" + group: tls name: tls_protocol_version type: string tls.resumed: @@ -2608,6 +2923,7 @@ entries: column: Bool examples: - true + group: tls name: tls_resumed type: boolean tls.server.certificate: @@ -2616,6 +2932,7 @@ entries: column: String examples: - MII... + group: tls name: tls_server_certificate type: string tls.server.certificate_chain: @@ -2625,6 +2942,7 @@ entries: examples: - MII... - MI... + group: tls name: tls_server_certificate_chain type: string[] tls.server.hash.md5: @@ -2633,6 +2951,7 @@ entries: column: String examples: - 0F76C7F2C55BFD7D8E8B8F4BFBF0C9EC + group: tls name: tls_server_hash_md5 type: string tls.server.hash.sha1: @@ -2641,6 +2960,7 @@ entries: column: String examples: - 9E393D93138888D288266C2D915214D1D1CCEB2A + group: tls name: tls_server_hash_sha1 type: string tls.server.hash.sha256: @@ -2649,6 +2969,7 @@ entries: column: String examples: - 0687F666A054EF17A08E2F2162EAB4CBC0D265E1D7875BE74BF3C712CA92DAF0 + group: tls name: tls_server_hash_sha256 type: string tls.server.issuer: @@ -2657,6 +2978,7 @@ entries: column: String examples: - CN=Example Root CA, OU=Infrastructure Team, DC=example, DC=com + group: tls name: tls_server_issuer type: string tls.server.ja3s: @@ -2664,6 +2986,7 @@ entries: column: String examples: - d4e5b18d6b55c71272893221c96ba240 + group: tls name: tls_server_ja3s type: string tls.server.not_after: @@ -2671,6 +2994,7 @@ entries: column: String examples: - "2021-01-01T00:00:00.000Z" + group: tls name: tls_server_not_after type: string tls.server.not_before: @@ -2678,6 +3002,7 @@ entries: column: String examples: - "1970-01-01T00:00:00.000Z" + group: tls name: tls_server_not_before type: string tls.server.subject: @@ -2686,6 +3011,7 @@ entries: column: String examples: - CN=myserver, OU=Documentation Team, DC=example, DC=com + group: tls name: tls_server_subject type: string url.fragment: @@ -2694,6 +3020,7 @@ entries: column: String examples: - SemConv + group: url name: url_fragment type: string url.full: @@ -2702,6 +3029,7 @@ entries: examples: - https://www.foo.bar/search?q=OpenTelemetry#SemConv - //localhost + group: url name: url_full type: string url.path: @@ -2709,6 +3037,7 @@ entries: column: String examples: - /search + group: url name: url_path type: string url.query: @@ -2716,6 +3045,7 @@ entries: column: String examples: - q=OpenTelemetry + group: url name: url_query type: string url.scheme: @@ -2726,6 +3056,7 @@ entries: - https - ftp - telnet + group: url name: url_scheme type: string user_agent.original: @@ -2736,6 +3067,7 @@ entries: - CERN-LineMode/2.15 libwww/2.17b3 - Mozilla/5.0 (iPhone; CPU iPhone OS 14_7_1 like Mac OS X) AppleWebKit/605.1.15 (KHTML, like Gecko) Version/14.1.2 Mobile/15E148 Safari/604.1 + group: user_agent name: user_agent_original type: string webengine.description: @@ -2744,6 +3076,7 @@ entries: column: String examples: - WildFly Full 21.0.0.Final (WildFly Core 13.0.1.Final) - 2.2.2.Final + group: webengine name: webengine_description type: string webengine.name: @@ -2752,6 +3085,7 @@ entries: column: String examples: - WildFly + group: webengine name: webengine_name type: string webengine.version: @@ -2760,10 +3094,11 @@ entries: column: String examples: - 21.0.0 + group: webengine name: webengine_version type: string statistics: deprecated: 24 enum: 23 - total: 668 + total: 334 unknown: 0 diff --git a/internal/otelschema/registry_test.go b/internal/otelschema/registry_test.go index 0f3b9b30..1331f11f 100644 --- a/internal/otelschema/registry_test.go +++ b/internal/otelschema/registry_test.go @@ -13,10 +13,54 @@ import ( "github.com/go-faster/sdk/gold" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "golang.org/x/exp/maps" "sigs.k8s.io/yaml" ) +func generateDDL(columns map[string]registryEntry) string { + var sb strings.Builder + sb.WriteString("CREATE TABLE columns (\n") + + groups := make(map[string][]registryEntry) + for _, c := range columns { + prefix := c.Group + groups[prefix] = append(groups[prefix], c) + } + + orderedGroups := maps.Keys(groups) + slices.Sort(orderedGroups) + + for i, groupName := range orderedGroups { + if groupName != "" { + sb.WriteString(fmt.Sprintf(" -- %s\n", groupName)) + } + maxFieldLen := 10 + for _, c := range groups[groupName] { + if len(c.Name) > maxFieldLen { + maxFieldLen = len(c.Name) + } + } + slices.SortFunc(groups[groupName], func(a, b registryEntry) int { + return strings.Compare(a.Name, b.Name) + }) + lastGroup := i == len(orderedGroups)-1 + for j, c := range groups[groupName] { + sb.WriteString(" ") + sb.WriteString(fmt.Sprintf("%-*s %s", maxFieldLen, c.Name, c.Column)) + sb.WriteString(fmt.Sprintf(" COMMENT '%s'", c.FullName)) + if !(lastGroup && j == len(groups[groupName])-1) { + sb.WriteString(",\n") + } + } + sb.WriteString("\n") + } + sb.WriteString(") ENGINE Null;") + return sb.String() +} + type registryEntry struct { + FullName string `json:"-"` + Group string `json:"group,omitempty"` Type string `json:"type"` Enum []any `json:"enum,omitempty"` Column proto.ColumnType `json:"column"` @@ -91,7 +135,7 @@ func columnType(name, brief, t string, enum []any) proto.ColumnType { var params []string for i, v := range anyTo[string](enum) { // Should we escape? - params = append(params, fmt.Sprintf("%d = '%s'", i, v)) + params = append(params, fmt.Sprintf("'%s' = %d", v, i)) } return colType.With(params...) } @@ -190,7 +234,13 @@ func TestParseAllAttributes(t *testing.T) { if len(enum) != 0 { examples = nil } + groupName := group.Prefix.Value + if i := strings.Index(name, "."); i != -1 { + groupName = name[:i] + } out.Entries[name] = registryEntry{ + FullName: name, + Group: groupName, Type: typ, Enum: enum, Column: columnType(name, v.Brief.Value, typ, enum), @@ -210,13 +260,13 @@ func TestParseAllAttributes(t *testing.T) { if strings.HasPrefix(e.Column.String(), "Enum") { out.Statistics.Enum++ } - out.Statistics.Total++ } data, err := yaml.Marshal(out) require.NoError(t, err) gold.Str(t, string(data), "registry.yaml") + gold.Str(t, generateDDL(out.Entries), "ddl.sql") assert.Zero(t, out.Statistics.Unknown, "Should be no unknown types") } From 3c6f24f5f5fb9c66b218466c5b56128b1d59c5b6 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 17:28:30 +0300 Subject: [PATCH 060/112] feat(otelschema): add table generator --- integration/chotele2e/e2e_test.go | 48 ++++--- integration/chotele2e/otelschema_test.go | 48 +++++++ integration/lokie2e/yt_test.go | 5 +- integration/prome2e/ch_test.go | 6 +- integration/prome2e/oas_test.go | 6 +- integration/skip.go | 13 ++ integration/tempoe2e/ch_test.go | 6 +- integration/tempoe2e/yt_test.go | 6 +- internal/otelschema/table.go | 152 +++++++++++++++++++++++ internal/otelschema/table_test.go | 22 ++++ 10 files changed, 274 insertions(+), 38 deletions(-) create mode 100644 integration/chotele2e/otelschema_test.go create mode 100644 integration/skip.go create mode 100644 internal/otelschema/table.go create mode 100644 internal/otelschema/table_test.go diff --git a/integration/chotele2e/e2e_test.go b/integration/chotele2e/e2e_test.go index 2de196cb..14aff6b7 100644 --- a/integration/chotele2e/e2e_test.go +++ b/integration/chotele2e/e2e_test.go @@ -4,7 +4,6 @@ import ( "context" "fmt" "math/rand" - "os" "strings" "sync" "testing" @@ -21,6 +20,7 @@ import ( "go.opentelemetry.io/otel/trace" "go.uber.org/zap" + "github.com/go-faster/oteldb/integration" "github.com/go-faster/oteldb/internal/chtrace" ) @@ -51,11 +51,9 @@ func discardResult() proto.Result { return (&proto.Results{}).Auto() } -func TestIntegrationTrace(t *testing.T) { - t.Parallel() - if os.Getenv("E2E") == "" { - t.Skip("Set E2E env to run") - } +func ConnectOpt(t *testing.T, connOpt ch.Options) *ch.Client { + t.Helper() + integration.Skip(t) ctx := context.Background() req := testcontainers.ContainerRequest{ @@ -78,6 +76,29 @@ func TestIntegrationTrace(t *testing.T) { connectBackoff.InitialInterval = 2 * time.Second connectBackoff.MaxElapsedTime = time.Minute + connOpt.Address = endpoint + conn, err := backoff.RetryWithData(func() (*ch.Client, error) { + c, err := ch.Dial(ctx, connOpt) + if err != nil { + return nil, errors.Wrap(err, "dial") + } + return c, nil + }, connectBackoff) + if err != nil { + t.Fatal(err) + } + return conn +} + +func Connect(t *testing.T) *ch.Client { + t.Helper() + return ConnectOpt(t, ch.Options{ + Logger: zap.NewNop(), + }) +} + +func TestIntegrationTrace(t *testing.T) { + ctx := context.Background() exporter := tracetest.NewInMemoryExporter() randSource := rand.NewSource(15) tp := tracesdk.NewTracerProvider( @@ -89,8 +110,7 @@ func TestIntegrationTrace(t *testing.T) { tracesdk.WithBatchTimeout(0), // instant ), ) - connOpt := ch.Options{ - Address: endpoint, + conn := ConnectOpt(t, ch.Options{ Logger: zap.NewNop(), OpenTelemetryInstrumentation: true, TracerProvider: tp, @@ -101,17 +121,7 @@ func TestIntegrationTrace(t *testing.T) { Important: true, }, }, - } - conn, err := backoff.RetryWithData(func() (*ch.Client, error) { - c, err := ch.Dial(ctx, connOpt) - if err != nil { - return nil, errors.Wrap(err, "dial") - } - return c, nil - }, connectBackoff) - if err != nil { - t.Fatal(err) - } + }) // Should record trace and spans. var traceID trace.TraceID diff --git a/integration/chotele2e/otelschema_test.go b/integration/chotele2e/otelschema_test.go new file mode 100644 index 00000000..22060395 --- /dev/null +++ b/integration/chotele2e/otelschema_test.go @@ -0,0 +1,48 @@ +package chotele2e + +import ( + "context" + "strings" + "testing" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/proto" + "github.com/stretchr/testify/require" + + "github.com/go-faster/oteldb/internal/otelschema" +) + +func TestSchema(t *testing.T) { + table := otelschema.NewTable([]otelschema.AttributeInfo{ + { + Name: "http.request.method", + Type: proto.ColumnTypeString, + }, + { + Name: "http.response.body.size", + Type: proto.ColumnTypeInt64, + }, + }) + require.NotEmpty(t, table.Columns) + + var ddl strings.Builder + ddl.WriteString("CREATE TABLE columns (") + for _, c := range table.Columns { + ddl.WriteString(c.Name) + ddl.WriteString(" ") + ddl.WriteString(c.Column.Type().String()) + ddl.WriteString(", ") + } + ddl.WriteString("raw String)") + ddl.WriteString(" ENGINE Null") + + c := Connect(t) + ctx := context.Background() + require.NoError(t, c.Do(ctx, ch.Query{Body: ddl.String()})) + + input := table.Input() + require.NoError(t, c.Do(ctx, ch.Query{ + Body: input.Into("columns"), + Input: input, + })) +} diff --git a/integration/lokie2e/yt_test.go b/integration/lokie2e/yt_test.go index de5c521e..6a0c65ac 100644 --- a/integration/lokie2e/yt_test.go +++ b/integration/lokie2e/yt_test.go @@ -17,15 +17,14 @@ import ( "go.ytsaurus.tech/yt/go/yt" "go.ytsaurus.tech/yt/go/yt/ythttp" + "github.com/go-faster/oteldb/integration" "github.com/go-faster/oteldb/internal/yqlclient" "github.com/go-faster/oteldb/internal/ytstorage" ) func TestYT(t *testing.T) { t.Parallel() - if os.Getenv("E2E") == "" { - t.Skip("Set E2E env to run") - } + integration.Skip(t) ctx := context.Background() req := testcontainers.ContainerRequest{ diff --git a/integration/prome2e/ch_test.go b/integration/prome2e/ch_test.go index 37de5ca7..aa462537 100644 --- a/integration/prome2e/ch_test.go +++ b/integration/prome2e/ch_test.go @@ -2,7 +2,6 @@ package prome2e_test import ( "context" - "os" "strings" "testing" "time" @@ -15,14 +14,13 @@ import ( "github.com/stretchr/testify/require" "github.com/testcontainers/testcontainers-go" + "github.com/go-faster/oteldb/integration" "github.com/go-faster/oteldb/internal/chstorage" ) func TestCH(t *testing.T) { t.Parallel() - if os.Getenv("E2E") == "" { - t.Skip("Set E2E env to run") - } + integration.Skip(t) ctx := context.Background() req := testcontainers.ContainerRequest{ diff --git a/integration/prome2e/oas_test.go b/integration/prome2e/oas_test.go index 5e9d9d67..51842a78 100644 --- a/integration/prome2e/oas_test.go +++ b/integration/prome2e/oas_test.go @@ -17,6 +17,7 @@ import ( "github.com/testcontainers/testcontainers-go" "github.com/testcontainers/testcontainers-go/wait" + "github.com/go-faster/oteldb/integration" "github.com/go-faster/oteldb/internal/promapi" ) @@ -38,10 +39,7 @@ func printJSON(t *testing.T, v jxEncode, name string) { var prometheusConfig []byte func TestPrometheusOAS(t *testing.T) { - if os.Getenv("E2E") == "" { - t.Skip("Set E2E env to run") - } - + integration.Skip(t) ctx := context.Background() // Provide config to prometheus testcontainer. diff --git a/integration/skip.go b/integration/skip.go new file mode 100644 index 00000000..d7560e24 --- /dev/null +++ b/integration/skip.go @@ -0,0 +1,13 @@ +package integration + +import ( + "os" + "testing" +) + +func Skip(t testing.TB) { + t.Helper() + if os.Getenv("E2E") == "" { + t.Skip("Set E2E env to run") + } +} diff --git a/integration/tempoe2e/ch_test.go b/integration/tempoe2e/ch_test.go index 28b99c8e..791af630 100644 --- a/integration/tempoe2e/ch_test.go +++ b/integration/tempoe2e/ch_test.go @@ -2,7 +2,6 @@ package tempoe2e_test import ( "context" - "os" "strings" "testing" "time" @@ -15,14 +14,13 @@ import ( "github.com/stretchr/testify/require" "github.com/testcontainers/testcontainers-go" + "github.com/go-faster/oteldb/integration" "github.com/go-faster/oteldb/internal/chstorage" ) func TestCH(t *testing.T) { t.Parallel() - if os.Getenv("E2E") == "" { - t.Skip("Set E2E env to run") - } + integration.Skip(t) ctx := context.Background() req := testcontainers.ContainerRequest{ diff --git a/integration/tempoe2e/yt_test.go b/integration/tempoe2e/yt_test.go index 2f39ec59..229b934c 100644 --- a/integration/tempoe2e/yt_test.go +++ b/integration/tempoe2e/yt_test.go @@ -2,7 +2,6 @@ package tempoe2e_test import ( "context" - "os" "testing" "time" @@ -16,14 +15,13 @@ import ( "go.ytsaurus.tech/yt/go/yt" "go.ytsaurus.tech/yt/go/yt/ythttp" + "github.com/go-faster/oteldb/integration" "github.com/go-faster/oteldb/internal/ytstorage" ) func TestYT(t *testing.T) { t.Parallel() - if os.Getenv("E2E") == "" { - t.Skip("Set E2E env to run") - } + integration.Skip(t) ctx := context.Background() req := testcontainers.ContainerRequest{ diff --git a/internal/otelschema/table.go b/internal/otelschema/table.go new file mode 100644 index 00000000..4ecee98a --- /dev/null +++ b/internal/otelschema/table.go @@ -0,0 +1,152 @@ +package otelschema + +import ( + "strings" + + "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/jx" + "go.opentelemetry.io/collector/pdata/pcommon" + + "github.com/go-faster/oteldb/internal/otelstorage" +) + +// Appendable is a column that can append values. +type Appendable interface { + AppendAttribute(v pcommon.Value) + AppendZero() +} + +// AppendableColumn is a column that can append values. +type AppendableColumn interface { + Appendable + proto.Column +} + +// TypedColumn is a column that can append values of a given type. +type TypedColumn[T any] struct { + proto.ColumnOf[T] + Append func(col proto.ColumnOf[T], v pcommon.Value) +} + +// AppendAttribute appends an attribute to the column. +func (t *TypedColumn[T]) AppendAttribute(v pcommon.Value) { + t.Append(t.ColumnOf, v) +} + +// AppendZero appends a zero value to the column. +func (t *TypedColumn[T]) AppendZero() { + var v T + t.ColumnOf.Append(v) +} + +// AttributeColumn is a column with a normalized name. +type AttributeColumn struct { + Key string + Name string + Column AppendableColumn +} + +// Table is a table with normalized column names. +type Table struct { + encoder jx.Encoder + + // Raw json. + Raw proto.ColStr + // Columns normalized. + Columns []AttributeColumn +} + +// Append appends attributes to the table. +func (t *Table) Append(attrs otelstorage.Attrs) { + kv := attrs.AsMap() + t.encoder.Reset() + + appendZeroes := make(map[string]struct{}, len(t.Columns)) + for _, c := range t.Columns { + appendZeroes[c.Key] = struct{}{} + } + + kv.Range(func(k string, v pcommon.Value) bool { + for _, c := range t.Columns { + if c.Key == k { + c.Column.AppendAttribute(v) + delete(appendZeroes, k) + return true + } + } + t.encoder.Field(k, func(e *jx.Encoder) { + switch v.Type() { + case pcommon.ValueTypeStr: + e.Str(v.Str()) + case pcommon.ValueTypeInt: + e.Int64(v.Int()) + default: + e.Null() + } + }) + return true + }) + t.Raw.AppendBytes(t.encoder.Bytes()) + for _, c := range t.Columns { + if _, ok := appendZeroes[c.Key]; !ok { + continue + } + c.Column.AppendZero() + } +} + +func newTypedColumn(t proto.ColumnType) AppendableColumn { + switch t { + case proto.ColumnTypeString: + return &TypedColumn[string]{ + ColumnOf: new(proto.ColStr), + Append: func(col proto.ColumnOf[string], v pcommon.Value) { col.Append(v.Str()) }, + } + case proto.ColumnTypeInt64: + return &TypedColumn[int64]{ + ColumnOf: new(proto.ColInt64), + Append: func(col proto.ColumnOf[int64], v pcommon.Value) { col.Append(v.Int()) }, + } + default: + panic("unknown column type") + } +} + +// Input returns input for a ClickHouse query. +func (t *Table) Input() proto.Input { + out := proto.Input{ + { + Name: "raw", + Data: &t.Raw, + }, + } + for _, c := range t.Columns { + out = append(out, proto.InputColumn{ + Name: c.Name, + Data: c.Column, + }) + } + return out +} + +// AttributeInfo describes a column. +type AttributeInfo struct { + Name string // col.name + Type proto.ColumnType +} + +// NewTable creates a new table with normalized column names. +func NewTable(columns []AttributeInfo) *Table { + t := &Table{ + Raw: proto.ColStr{}, + } + for _, c := range columns { + mappedName := strings.ReplaceAll(c.Name, ".", "_") + t.Columns = append(t.Columns, AttributeColumn{ + Key: c.Name, + Name: mappedName, + Column: newTypedColumn(c.Type), + }) + } + return t +} diff --git a/internal/otelschema/table_test.go b/internal/otelschema/table_test.go new file mode 100644 index 00000000..104c2f4b --- /dev/null +++ b/internal/otelschema/table_test.go @@ -0,0 +1,22 @@ +package otelschema + +import ( + "testing" + + "github.com/ClickHouse/ch-go/proto" + "github.com/stretchr/testify/require" +) + +func TestNewTable(t *testing.T) { + table := NewTable([]AttributeInfo{ + { + Name: "http.request.method", + Type: proto.ColumnTypeString, + }, + { + Name: "http.response.body.size", + Type: proto.ColumnTypeInt64, + }, + }) + require.NotEmpty(t, table.Columns) +} From fcacd80100e24470b41da204f2bb88e2ee03ce06 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Sun, 26 Nov 2023 17:38:52 +0300 Subject: [PATCH 061/112] fix(otelschema): proper json encoding --- integration/chotele2e/otelschema_test.go | 19 +++++++++++- internal/otelschema/table.go | 11 +++++++ internal/otelschema/table_test.go | 39 ++++++++++++++++++++++++ 3 files changed, 68 insertions(+), 1 deletion(-) diff --git a/integration/chotele2e/otelschema_test.go b/integration/chotele2e/otelschema_test.go index 22060395..c16fb5bd 100644 --- a/integration/chotele2e/otelschema_test.go +++ b/integration/chotele2e/otelschema_test.go @@ -7,9 +7,13 @@ import ( "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/jx" "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.uber.org/zap/zaptest" "github.com/go-faster/oteldb/internal/otelschema" + "github.com/go-faster/oteldb/internal/otelstorage" ) func TestSchema(t *testing.T) { @@ -36,11 +40,24 @@ func TestSchema(t *testing.T) { ddl.WriteString("raw String)") ddl.WriteString(" ENGINE Null") - c := Connect(t) + c := ConnectOpt(t, ch.Options{ + Logger: zaptest.NewLogger(t), + }) ctx := context.Background() require.NoError(t, c.Do(ctx, ch.Query{Body: ddl.String()})) input := table.Input() + m := pcommon.NewMap() + m.PutStr("http.request.method", "GET") + m.PutInt("http.response.body.size", 123) + m.PutStr("service.name", "foo") + table.Append(otelstorage.Attrs(m)) + + require.Equal(t, 1, table.Raw.Rows()) + data := table.Raw.Row(0) + require.True(t, jx.Valid([]byte(data))) + t.Log(data) + require.NoError(t, c.Do(ctx, ch.Query{ Body: input.Into("columns"), Input: input, diff --git a/internal/otelschema/table.go b/internal/otelschema/table.go index 4ecee98a..977760b5 100644 --- a/internal/otelschema/table.go +++ b/internal/otelschema/table.go @@ -56,6 +56,15 @@ type Table struct { Columns []AttributeColumn } +// Reset resets table columns. +func (t *Table) Reset() { + t.encoder.Reset() + t.Raw.Reset() + for _, c := range t.Columns { + c.Column.Reset() + } +} + // Append appends attributes to the table. func (t *Table) Append(attrs otelstorage.Attrs) { kv := attrs.AsMap() @@ -66,6 +75,7 @@ func (t *Table) Append(attrs otelstorage.Attrs) { appendZeroes[c.Key] = struct{}{} } + t.encoder.ObjStart() kv.Range(func(k string, v pcommon.Value) bool { for _, c := range t.Columns { if c.Key == k { @@ -86,6 +96,7 @@ func (t *Table) Append(attrs otelstorage.Attrs) { }) return true }) + t.encoder.ObjEnd() t.Raw.AppendBytes(t.encoder.Bytes()) for _, c := range t.Columns { if _, ok := appendZeroes[c.Key]; !ok { diff --git a/internal/otelschema/table_test.go b/internal/otelschema/table_test.go index 104c2f4b..d7260aab 100644 --- a/internal/otelschema/table_test.go +++ b/internal/otelschema/table_test.go @@ -4,7 +4,11 @@ import ( "testing" "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/jx" "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/pdata/pcommon" + + "github.com/go-faster/oteldb/internal/otelstorage" ) func TestNewTable(t *testing.T) { @@ -18,5 +22,40 @@ func TestNewTable(t *testing.T) { Type: proto.ColumnTypeInt64, }, }) + m := pcommon.NewMap() + m.PutStr("http.request.method", "GET") + m.PutInt("http.response.body.size", 123) + m.PutStr("service.name", "foo") + table.Append(otelstorage.Attrs(m)) + + require.Equal(t, 1, table.Raw.Rows()) + data := table.Raw.Row(0) + require.True(t, jx.Valid([]byte(data)), "invalid json: %s", data) + t.Log(data) require.NotEmpty(t, table.Columns) } + +func BenchmarkTable_Append(b *testing.B) { + table := NewTable([]AttributeInfo{ + { + Name: "http.request.method", + Type: proto.ColumnTypeString, + }, + { + Name: "http.response.body.size", + Type: proto.ColumnTypeInt64, + }, + }) + m := pcommon.NewMap() + m.PutStr("http.request.method", "GET") + m.PutInt("http.response.body.size", 123) + m.PutStr("service.name", "foo") + + b.ResetTimer() + b.ReportAllocs() + + for i := 0; i < b.N; i++ { + table.Reset() + table.Append(otelstorage.Attrs(m)) + } +} From 7409588041f18aaa9a2bd10dee966f4c6b1b520c Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 27 Nov 2023 14:42:55 +0300 Subject: [PATCH 062/112] feat(chstorage): optimize schema --- integration/lokie2e/ch_test.go | 85 +++++++++++++++++++++++ integration/skip.go | 2 + internal/chstorage/columns.go | 40 +++++++++++ internal/chstorage/columns_logs.go | 106 +++++++++++++++++------------ internal/chstorage/schema_logs.go | 47 ++++++++----- 5 files changed, 222 insertions(+), 58 deletions(-) create mode 100644 integration/lokie2e/ch_test.go create mode 100644 internal/chstorage/columns.go diff --git a/integration/lokie2e/ch_test.go b/integration/lokie2e/ch_test.go new file mode 100644 index 00000000..55319e92 --- /dev/null +++ b/integration/lokie2e/ch_test.go @@ -0,0 +1,85 @@ +package lokie2e_test + +import ( + "context" + "strings" + "testing" + "time" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/chpool" + "github.com/cenkalti/backoff/v4" + "github.com/go-faster/errors" + "github.com/google/uuid" + "github.com/stretchr/testify/require" + "github.com/testcontainers/testcontainers-go" + + "github.com/go-faster/oteldb/integration" + "github.com/go-faster/oteldb/internal/chstorage" + "github.com/go-faster/oteldb/internal/logstorage" +) + +func TestCH(t *testing.T) { + t.Parallel() + integration.Skip(t) + ctx := context.Background() + + req := testcontainers.ContainerRequest{ + Name: "oteldb-lokie2e-clickhouse", + Image: "clickhouse/clickhouse-server:23.10", + ExposedPorts: []string{"8123/tcp", "9000/tcp"}, + } + chContainer, err := testcontainers.GenericContainer(ctx, testcontainers.GenericContainerRequest{ + ContainerRequest: req, + Started: true, + Logger: testcontainers.TestLogger(t), + Reuse: true, + }) + require.NoError(t, err, "container start") + + endpoint, err := chContainer.PortEndpoint(ctx, "9000", "") + require.NoError(t, err, "container endpoint") + + opts := ch.Options{ + Address: endpoint, + Database: "default", + } + + connectBackoff := backoff.NewExponentialBackOff() + connectBackoff.InitialInterval = 2 * time.Second + connectBackoff.MaxElapsedTime = time.Minute + c, err := backoff.RetryWithData(func() (*chpool.Pool, error) { + c, err := chpool.Dial(ctx, chpool.Options{ + ClientOptions: opts, + }) + if err != nil { + return nil, errors.Wrap(err, "dial") + } + return c, nil + }, connectBackoff) + if err != nil { + t.Fatal(err) + } + + prefix := strings.ReplaceAll(uuid.NewString(), "-", "") + tables := chstorage.DefaultTables() + tables.Each(func(name *string) error { + old := *name + *name = prefix + "_" + old + return nil + }) + t.Logf("Test tables prefix: %s", prefix) + require.NoError(t, tables.Create(ctx, c)) + + inserter, err := chstorage.NewInserter(c, chstorage.InserterOptions{Tables: tables}) + require.NoError(t, err) + + consumer := logstorage.NewConsumer(inserter) + set, err := readBatchSet("_testdata/logs.json") + require.NoError(t, err) + for i, b := range set.Batches { + if err := consumer.ConsumeLogs(ctx, b); err != nil { + t.Fatalf("Send batch %d: %+v", i, err) + } + } +} diff --git a/integration/skip.go b/integration/skip.go index d7560e24..1da92cab 100644 --- a/integration/skip.go +++ b/integration/skip.go @@ -1,3 +1,4 @@ +// Package integration wraps integration package integration import ( @@ -5,6 +6,7 @@ import ( "testing" ) +// Skip test if E2E env is not set. func Skip(t testing.TB) { t.Helper() if os.Getenv("E2E") == "" { diff --git a/internal/chstorage/columns.go b/internal/chstorage/columns.go new file mode 100644 index 00000000..da990e92 --- /dev/null +++ b/internal/chstorage/columns.go @@ -0,0 +1,40 @@ +package chstorage + +import "github.com/ClickHouse/ch-go/proto" + +type tableColumn struct { + Name string + Data proto.Column +} + +type tableColumns []tableColumn + +func (c tableColumns) Names() []string { + var names []string + for _, col := range c { + names = append(names, col.Name) + } + return names +} + +func (c tableColumns) Input() proto.Input { + var cols proto.Input + for _, col := range c { + cols = append(cols, proto.InputColumn{ + Name: col.Name, + Data: col.Data, + }) + } + return cols +} + +func (c tableColumns) Result() proto.Results { + var cols proto.Results + for _, col := range c { + cols = append(cols, proto.ResultColumn{ + Name: col.Name, + Data: col.Data, + }) + } + return cols +} diff --git a/internal/chstorage/columns_logs.go b/internal/chstorage/columns_logs.go index 5cbb4036..08116220 100644 --- a/internal/chstorage/columns_logs.go +++ b/internal/chstorage/columns_logs.go @@ -2,32 +2,44 @@ package chstorage import ( "github.com/ClickHouse/ch-go/proto" + "go.opentelemetry.io/collector/pdata/pcommon" "github.com/go-faster/oteldb/internal/logstorage" "github.com/go-faster/oteldb/internal/otelstorage" ) type logColumns struct { - timestamp *proto.ColDateTime64 - observedTimestamp *proto.ColDateTime64 - flags proto.ColUInt32 - severityNumber proto.ColInt32 - severityText *proto.ColLowCardinality[string] - body proto.ColStr - traceID proto.ColRawOf[otelstorage.TraceID] - spanID proto.ColRawOf[otelstorage.SpanID] - attributes proto.ColStr - resource proto.ColStr - scopeName proto.ColStr - scopeVersion proto.ColStr - scopeAttributes proto.ColStr + serviceInstanceID *proto.ColLowCardinality[string] + serviceName *proto.ColLowCardinality[string] + serviceNamespace *proto.ColLowCardinality[string] + + timestamp *proto.ColDateTime64 + + severityText *proto.ColLowCardinality[string] + severityNumber proto.ColUInt8 + + traceFlags proto.ColUInt8 + traceID proto.ColRawOf[otelstorage.TraceID] + spanID proto.ColRawOf[otelstorage.SpanID] + + body proto.ColStr + attributes proto.ColStr + resource proto.ColStr + + scopeName *proto.ColLowCardinality[string] + scopeVersion *proto.ColLowCardinality[string] + scopeAttributes proto.ColStr } func newLogColumns() *logColumns { return &logColumns{ + serviceName: new(proto.ColStr).LowCardinality(), + serviceInstanceID: new(proto.ColStr).LowCardinality(), + serviceNamespace: new(proto.ColStr).LowCardinality(), timestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), - observedTimestamp: new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano), severityText: new(proto.ColStr).LowCardinality(), + scopeName: new(proto.ColStr).LowCardinality(), + scopeVersion: new(proto.ColStr).LowCardinality(), } } @@ -39,54 +51,64 @@ func (c *logColumns) StaticColumns() []string { return cols } +func setStrOrEmpty(col proto.ColumnOf[string], m pcommon.Map, k string) { + v, ok := m.Get(k) + if !ok { + col.Append("") + return + } + col.Append(v.AsString()) +} + func (c *logColumns) AddRow(r logstorage.Record) { + { + m := r.ResourceAttrs.AsMap() + setStrOrEmpty(c.serviceInstanceID, m, "service.instance.id") + setStrOrEmpty(c.serviceName, m, "service.name") + setStrOrEmpty(c.serviceNamespace, m, "service.namespace") + } c.timestamp.Append(r.Timestamp.AsTime()) - c.observedTimestamp.Append(r.ObservedTimestamp.AsTime()) - c.flags.Append(uint32(r.Flags)) - c.severityNumber.Append(int32(r.SeverityNumber)) + + c.severityNumber.Append(uint8(r.SeverityNumber)) c.severityText.Append(r.SeverityText) - c.body.Append(r.Body) + c.traceID.Append(r.TraceID) c.spanID.Append(r.SpanID) + c.traceFlags.Append(uint8(r.Flags)) + + c.body.Append(r.Body) c.attributes.Append(encodeAttributes(r.Attrs.AsMap())) c.resource.Append(encodeAttributes(r.ResourceAttrs.AsMap())) + c.scopeName.Append(r.ScopeName) c.scopeVersion.Append(r.ScopeVersion) c.scopeAttributes.Append(encodeAttributes(r.ScopeAttrs.AsMap())) } -func (c *logColumns) Input() proto.Input { - return proto.Input{ - {Name: "timestamp", Data: c.timestamp}, - {Name: "observed_timestamp", Data: c.observedTimestamp}, - {Name: "flags", Data: c.flags}, - {Name: "severity_number", Data: c.severityNumber}, - {Name: "severity_text", Data: c.severityText}, - {Name: "body", Data: c.body}, - {Name: "trace_id", Data: c.traceID}, - {Name: "span_id", Data: c.spanID}, - {Name: "attributes", Data: c.attributes}, - {Name: "resource", Data: c.resource}, - {Name: "scope_name", Data: c.scopeName}, - {Name: "scope_version", Data: c.scopeVersion}, - {Name: "scope_attributes", Data: c.scopeAttributes}, - } -} +func (c *logColumns) columns() tableColumns { + return []tableColumn{ + {Name: "service_instance_id", Data: c.serviceInstanceID}, + {Name: "service_name", Data: c.serviceName}, + {Name: "service_namespace", Data: c.serviceNamespace}, -func (c *logColumns) Result() proto.Results { - return proto.Results{ {Name: "timestamp", Data: c.timestamp}, - {Name: "observed_timestamp", Data: c.observedTimestamp}, - {Name: "flags", Data: &c.flags}, + {Name: "severity_number", Data: &c.severityNumber}, {Name: "severity_text", Data: c.severityText}, - {Name: "body", Data: &c.body}, + {Name: "trace_id", Data: &c.traceID}, {Name: "span_id", Data: &c.spanID}, + {Name: "trace_flags", Data: &c.traceFlags}, + + {Name: "body", Data: &c.body}, {Name: "attributes", Data: &c.attributes}, {Name: "resource", Data: &c.resource}, - {Name: "scope_name", Data: &c.scopeName}, - {Name: "scope_version", Data: &c.scopeVersion}, + + {Name: "scope_name", Data: c.scopeName}, + {Name: "scope_version", Data: c.scopeVersion}, {Name: "scope_attributes", Data: &c.scopeAttributes}, } } + +func (c *logColumns) Input() proto.Input { return c.columns().Input() } +func (c *logColumns) Result() proto.Results { return c.columns().Result() } diff --git a/internal/chstorage/schema_logs.go b/internal/chstorage/schema_logs.go index ece8d9a8..ab76a90b 100644 --- a/internal/chstorage/schema_logs.go +++ b/internal/chstorage/schema_logs.go @@ -1,22 +1,37 @@ package chstorage const ( - logsSchema = `CREATE TABLE IF NOT EXISTS %s + logsSchema = ` +-- https://opentelemetry.io/docs/specs/otel/logs/data-model/#log-and-event-record-definition +CREATE TABLE IF NOT EXISTS %s ( - timestamp DateTime64(9), - observed_timestamp DateTime64(9), - flags UInt32, - severity_text String, - severity_number Int32, - body String, - trace_id FixedString(16), - span_id FixedString(8), - attributes String, - resource String, - scope_name String, - scope_version String, - scope_attributes String, + -- materialized fields from semantic conventions + -- NB: They MUST NOT be present in the 'resource' field. + service_instance_id LowCardinality(String) COMMENT 'service.instance.id', + service_name LowCardinality(String) COMMENT 'service.name', + service_namespace LowCardinality(String) COMMENT 'service.namespace', + + -- Timestamp, or ObservedTimestamp if not present. + timestamp DateTime64(9) CODEC(DoubleDelta), + + -- Severity Fields + severity_text LowCardinality(String), -- SeverityText + severity_number UInt8, -- SeverityNumber [1, 24] + + -- Trace Context Fields + trace_id FixedString(16), -- TraceId + span_id FixedString(8), -- SpanId + trace_flags UInt8, -- TraceFlags + + -- can be arbitrary json + body String, -- json + attributes String, -- json object + resource String, -- json object + + scope_name LowCardinality(String), + scope_version LowCardinality(String), + scope_attributes String, -- json object ) -ENGINE = MergeTree() -ORDER BY (timestamp);` + ENGINE = MergeTree() + ORDER BY (service_namespace, service_name, service_instance_id, toStartOfFiveMinutes(timestamp));` ) From 1c690f727234d739380ba9d523c0e3afd5be2b29 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 27 Nov 2023 16:58:08 +0300 Subject: [PATCH 063/112] feat(chstorage): draft logs impl --- README.md | 2 +- cmd/oteldb/storage.go | 6 +- cmd/otelfaker/main.go | 14 +- integration/lokie2e/ch_test.go | 15 +- internal/chstorage/columns_logs.go | 62 ++++ internal/chstorage/escape.go | 2 +- internal/chstorage/querier_logs.go | 270 ++++++++++++++++++ internal/logql/logqlengine/engine.go | 4 +- internal/logql/logqlengine/engine_test.go | 2 +- internal/logql/logqlengine/precondition.go | 2 +- .../logql/logqlengine/precondition_test.go | 4 +- internal/logql/logqlengine/storage.go | 10 +- internal/ytstorage/yql_querier_logs.go | 4 +- internal/ytstorage/ytql_querier_logs.go | 4 +- 14 files changed, 368 insertions(+), 33 deletions(-) create mode 100644 internal/chstorage/querier_logs.go diff --git a/README.md b/README.md index bd96fabc..60e7f81e 100644 --- a/README.md +++ b/README.md @@ -46,5 +46,5 @@ docker compose -f dev/local/ytsaurus/docker-compose.yml up -d #### Clickhouse storage ```shell -docker compose -f dev/local/clickhouse/docker-compose.yml up -d +docker compose -f dev/local/ch/docker-compose.yml up -d ``` diff --git a/cmd/oteldb/storage.go b/cmd/oteldb/storage.go index e9772463..8db9d2ad 100644 --- a/cmd/oteldb/storage.go +++ b/cmd/oteldb/storage.go @@ -40,8 +40,8 @@ type combinedYTQuerier struct { *ytstorage.YTQLQuerier } -func (q *combinedYTQuerier) Сapabilities() (caps logqlengine.QuerierСapabilities) { - return q.yql.Сapabilities() +func (q *combinedYTQuerier) Capabilities() (caps logqlengine.QuerierCapabilities) { + return q.yql.Capabilities() } func (q *combinedYTQuerier) SelectLogs(ctx context.Context, start, end otelstorage.Timestamp, params logqlengine.SelectLogsParams) (_ iterators.Iterator[logstorage.Record], rerr error) { @@ -288,7 +288,7 @@ func setupCH( } return otelStorage{ - logQuerier: nil, + logQuerier: querier, logInserter: inserter, traceQuerier: querier, traceInserter: inserter, diff --git a/cmd/otelfaker/main.go b/cmd/otelfaker/main.go index 60731de7..c1e3791a 100644 --- a/cmd/otelfaker/main.go +++ b/cmd/otelfaker/main.go @@ -2,6 +2,7 @@ package main import ( "context" + "math/rand" "os" "time" @@ -18,7 +19,7 @@ import ( "google.golang.org/grpc/credentials/insecure" ) -func getLogs(ctx context.Context, tracer trace.Tracer, now time.Time) plog.Logs { +func getLogs(ctx context.Context, tracer trace.Tracer, rnd *rand.Rand, now time.Time) plog.Logs { _, span := tracer.Start(ctx, "getLogs") defer span.End() var ( @@ -49,8 +50,12 @@ func getLogs(ctx context.Context, tracer trace.Tracer, now time.Time) plog.Logs lg.Attributes().PutStr("http.method", "GET") lg.Attributes().PutBool("http.server", true) lg.Attributes().PutInt("http.status_code", 200) - lg.Attributes().PutStr("http.url", "https://example.com") - lg.Attributes().PutStr("http.user_agent", "test-agent") + if rnd.Float32() < 0.5 { + lg.Attributes().PutStr("http.url", "https://example.com") + lg.Attributes().PutStr("http.status_text", "OK") + } else { + lg.Attributes().PutStr("http.user_agent", "test-agent") + } lg.Attributes().PutDouble("http.duration_seconds", 1.1054) lg.Attributes().PutInt("http.duration", (time.Second + time.Millisecond*105).Nanoseconds()) lg.SetFlags(plog.DefaultLogRecordFlags.WithIsSampled(true)) @@ -77,8 +82,9 @@ func main() { } client := plogotlp.NewGRPCClient(conn) tracer := m.TracerProvider().Tracer("otelfaker") + rnd := rand.New(rand.NewSource(time.Now().UnixNano())) // #nosec G404 for now := range time.NewTicker(time.Second).C { - if _, err := client.Export(ctx, plogotlp.NewExportRequestFromLogs(getLogs(ctx, tracer, now))); err != nil { + if _, err := client.Export(ctx, plogotlp.NewExportRequestFromLogs(getLogs(ctx, tracer, rnd, now))); err != nil { return errors.Wrap(err, "send logs") } } diff --git a/integration/lokie2e/ch_test.go b/integration/lokie2e/ch_test.go index 55319e92..b7af8dfa 100644 --- a/integration/lokie2e/ch_test.go +++ b/integration/lokie2e/ch_test.go @@ -10,17 +10,17 @@ import ( "github.com/ClickHouse/ch-go/chpool" "github.com/cenkalti/backoff/v4" "github.com/go-faster/errors" + "github.com/go-faster/sdk/zctx" "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/testcontainers/testcontainers-go" + "go.uber.org/zap/zaptest" "github.com/go-faster/oteldb/integration" "github.com/go-faster/oteldb/internal/chstorage" - "github.com/go-faster/oteldb/internal/logstorage" ) func TestCH(t *testing.T) { - t.Parallel() integration.Skip(t) ctx := context.Background() @@ -74,12 +74,9 @@ func TestCH(t *testing.T) { inserter, err := chstorage.NewInserter(c, chstorage.InserterOptions{Tables: tables}) require.NoError(t, err) - consumer := logstorage.NewConsumer(inserter) - set, err := readBatchSet("_testdata/logs.json") + querier, err := chstorage.NewQuerier(c, chstorage.QuerierOptions{Tables: tables}) require.NoError(t, err) - for i, b := range set.Batches { - if err := consumer.ConsumeLogs(ctx, b); err != nil { - t.Fatalf("Send batch %d: %+v", i, err) - } - } + + ctx = zctx.Base(ctx, zaptest.NewLogger(t)) + runTest(ctx, t, inserter, querier, querier) } diff --git a/internal/chstorage/columns_logs.go b/internal/chstorage/columns_logs.go index 08116220..12aad13f 100644 --- a/internal/chstorage/columns_logs.go +++ b/internal/chstorage/columns_logs.go @@ -2,7 +2,9 @@ package chstorage import ( "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/errors" "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/plog" "github.com/go-faster/oteldb/internal/logstorage" "github.com/go-faster/oteldb/internal/otelstorage" @@ -60,6 +62,60 @@ func setStrOrEmpty(col proto.ColumnOf[string], m pcommon.Map, k string) { col.Append(v.AsString()) } +func (c *logColumns) ForEach(f func(r logstorage.Record)) error { + for i := 0; i < c.timestamp.Rows(); i++ { + r := logstorage.Record{ + Timestamp: otelstorage.NewTimestampFromTime(c.timestamp.Row(i)), + SeverityText: c.severityText.Row(i), + SeverityNumber: plog.SeverityNumber(c.severityNumber.Row(i)), + TraceID: c.traceID.Row(i), + SpanID: c.spanID.Row(i), + Flags: plog.LogRecordFlags(c.traceFlags.Row(i)), + Body: c.body.Row(i), + + ScopeVersion: c.scopeVersion.Row(i), + ScopeName: c.scopeName.Row(i), + } + { + m, err := decodeAttributes(c.resource.Row(i)) + if err != nil { + return errors.Wrap(err, "decode resource") + } + v := m.AsMap() + if s := c.serviceInstanceID.Row(i); s != "" { + v.PutStr("service.instance.id", s) + } + if s := c.serviceName.Row(i); s != "" { + v.PutStr("service.name", s) + } + if s := c.serviceNamespace.Row(i); s != "" { + v.PutStr("service.namespace", s) + } + r.ResourceAttrs = otelstorage.Attrs(v) + } + { + m, err := decodeAttributes(c.attributes.Row(i)) + if err != nil { + return errors.Wrap(err, "decode attributes") + } + r.Attrs = otelstorage.Attrs(m.AsMap()) + } + { + m, err := decodeAttributes(c.scopeAttributes.Row(i)) + if err != nil { + return errors.Wrap(err, "decode scope attributes") + } + r.ScopeAttrs = otelstorage.Attrs(m.AsMap()) + } + { + // Default just to timestamp. + r.ObservedTimestamp = r.Timestamp + } + f(r) + } + return nil +} + func (c *logColumns) AddRow(r logstorage.Record) { { m := r.ResourceAttrs.AsMap() @@ -112,3 +168,9 @@ func (c *logColumns) columns() tableColumns { func (c *logColumns) Input() proto.Input { return c.columns().Input() } func (c *logColumns) Result() proto.Results { return c.columns().Result() } + +func (c *logColumns) Reset() { + for _, col := range c.columns() { + col.Data.Reset() + } +} diff --git a/internal/chstorage/escape.go b/internal/chstorage/escape.go index 37851ab8..3c758155 100644 --- a/internal/chstorage/escape.go +++ b/internal/chstorage/escape.go @@ -6,7 +6,7 @@ import ( "unicode/utf8" ) -func singleQuoted[S string | []byte](s S) string { +func singleQuoted[S ~[]byte | ~string](s S) string { const lowerhex = "0123456789abcdef" var sb strings.Builder diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go new file mode 100644 index 00000000..7c80e153 --- /dev/null +++ b/internal/chstorage/querier_logs.go @@ -0,0 +1,270 @@ +package chstorage + +import ( + "context" + "fmt" + "strings" + + "github.com/ClickHouse/ch-go" + "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/errors" + "github.com/go-faster/jx" + "github.com/go-faster/sdk/zctx" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/trace" + + "github.com/go-faster/oteldb/internal/iterators" + "github.com/go-faster/oteldb/internal/logql" + "github.com/go-faster/oteldb/internal/logql/logqlengine" + "github.com/go-faster/oteldb/internal/logstorage" + "github.com/go-faster/oteldb/internal/otelstorage" +) + +var _ logstorage.Querier = (*Querier)(nil) +var _ logqlengine.Querier = (*Querier)(nil) + +// LabelNames implements logstorage.Querier. +func (q *Querier) LabelNames(ctx context.Context, opts logstorage.LabelsOptions) (_ []string, rerr error) { + table := q.tables.Logs + + ctx, span := q.tracer.Start(ctx, "LabelNames", + trace.WithAttributes( + attribute.Int64("chstorage.start_range", int64(opts.Start)), + attribute.Int64("chstorage.end_range", int64(opts.End)), + attribute.String("chstorage.table", table), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + var ( + names proto.ColStr + out []string + ) + if err := q.ch.Do(ctx, ch.Query{ + Logger: zctx.From(ctx).Named("ch"), + Result: proto.Results{ + {Name: "key", Data: &names}, + }, + OnResult: func(ctx context.Context, block proto.Block) error { + for i := 0; i < names.Rows(); i++ { + out = append(out, names.Row(i)) + } + return nil + }, + Body: fmt.Sprintf(`SELECT DISTINCT +arrayJoin(arrayConcat(JSONExtractKeys(attributes), JSONExtractKeys(resource), JSONExtractKeys(scope_attributes))) as key +FROM %s +WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d)`, + table, opts.Start, opts.End, + ), + }); err != nil { + return nil, errors.Wrap(err, "select") + } + + return out, nil +} + +type labelStaticIterator struct { + name string + values []jx.Raw +} + +func (l *labelStaticIterator) Next(t *logstorage.Label) bool { + if len(l.values) == 0 { + return false + } + t.Name = l.name + e := jx.DecodeBytes(l.values[0]) + switch e.Next() { + case jx.String: + t.Type = int32(pcommon.ValueTypeStr) + s, _ := e.Str() + t.Value = s + case jx.Number: + n, _ := e.Num() + if n.IsInt() { + t.Type = int32(pcommon.ValueTypeInt) + v, _ := n.Int64() + t.Value = fmt.Sprintf("%d", v) + } else { + t.Type = int32(pcommon.ValueTypeDouble) + v, _ := n.Float64() + t.Value = fmt.Sprintf("%f", v) + } + default: + t.Type = int32(pcommon.ValueTypeStr) + t.Value = l.values[0].String() + } + return true +} + +func (l *labelStaticIterator) Err() error { return nil } +func (l *labelStaticIterator) Close() error { return nil } + +// LabelValues implements logstorage.Querier. +func (q *Querier) LabelValues(ctx context.Context, labelName string, opts logstorage.LabelsOptions) (_ iterators.Iterator[logstorage.Label], rerr error) { + table := q.tables.Logs + + ctx, span := q.tracer.Start(ctx, "LabelValues", + trace.WithAttributes( + attribute.Int64("chstorage.start_range", int64(opts.Start)), + attribute.Int64("chstorage.end_range", int64(opts.End)), + attribute.String("chstorage.table", table), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + var ( + names proto.ColStr + out []jx.Raw + ) + if err := q.ch.Do(ctx, ch.Query{ + Logger: zctx.From(ctx).Named("ch"), + Result: proto.Results{ + {Name: "value", Data: &names}, + }, + OnResult: func(ctx context.Context, block proto.Block) error { + for i := 0; i < names.Rows(); i++ { + out = append(out, jx.Raw(names.Row(i))) + } + return nil + }, + Body: fmt.Sprintf(`SELECT DISTINCT +COALESCE( + JSONExtractRaw(attributes, %[1]s), + JSONExtractRaw(scope_attributes, %[1]s), + JSONExtractRaw(resource, %[1]s) +) as value +FROM %s +WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d)`, + singleQuoted(labelName), table, opts.Start, opts.End, + ), + }); err != nil { + return nil, errors.Wrap(err, "select") + } + + return &labelStaticIterator{ + name: labelName, + values: out, + }, nil +} + +// Capabilities implements logqlengine.Querier. +func (q *Querier) Capabilities() (caps logqlengine.QuerierCapabilities) { + caps.Label.Add(logql.OpEq, logql.OpNotEq, logql.OpRe, logql.OpNotRe) + caps.Line.Add(logql.OpEq, logql.OpNotEq, logql.OpRe, logql.OpNotRe) + return caps +} + +type logStaticIterator struct { + data []logstorage.Record +} + +func (l *logStaticIterator) Next(t *logstorage.Record) bool { + if len(l.data) == 0 { + return false + } + *t = l.data[0] + l.data = l.data[1:] + return true +} + +func (l *logStaticIterator) Err() error { return nil } +func (l *logStaticIterator) Close() error { return nil } + +// SelectLogs implements logqlengine.Querier. +func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timestamp, params logqlengine.SelectLogsParams) (_ iterators.Iterator[logstorage.Record], rerr error) { + table := q.tables.Logs + + ctx, span := q.tracer.Start(ctx, "SelectLogs", + trace.WithAttributes( + attribute.Int("chstorage.labels_count", len(params.Labels)), + attribute.Int64("chstorage.start_range", int64(start)), + attribute.Int64("chstorage.end_range", int64(end)), + attribute.String("chstorage.table", table), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + out := newLogColumns() + var query strings.Builder + query.WriteString("SELECT ") + for i, column := range out.StaticColumns() { + if i != 0 { + query.WriteByte(',') + } + query.WriteString(column) + } + fmt.Fprintf(&query, " FROM %s WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d)", table, start, end) + for _, m := range params.Labels { + switch m.Op { + case logql.OpEq: + query.WriteString(" AND (") + case logql.OpNotEq: + query.WriteString(" AND NOT (") + default: + return nil, errors.Errorf("unexpected op %q", m.Op) + } + for i, column := range []string{ + "attributes", + "resource", + "scope_attributes", + } { + if i != 0 { + query.WriteString(" OR ") + } + // TODO: how to match integers, booleans, floats, arrays? + fmt.Fprintf(&query, "JSONExtractString(%s, %s) = %s", column, singleQuoted(m.Label), singleQuoted(m.Value)) + } + query.WriteByte(')') + } + for _, m := range params.Line { + switch m.Op { + case logql.OpEq: + query.WriteString(" AND ") + case logql.OpNotEq: + query.WriteString(" AND NOT ") + default: + return nil, errors.Errorf("unexpected op %q", m.Op) + } + + // Line filter checks if line contains given value. + fmt.Fprintf(&query, "positionUTF8(body, %s) > 0", singleQuoted(m.Value)) + } + + // TODO: use streaming. + var data []logstorage.Record + if err := q.ch.Do(ctx, ch.Query{ + Logger: zctx.From(ctx).Named("ch"), + Body: query.String(), + Result: out.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + if err := out.ForEach(func(r logstorage.Record) { + data = append(data, r) + }); err != nil { + return errors.Wrap(err, "for each") + } + out.Reset() + return nil + }, + }); err != nil { + return nil, errors.Wrap(err, "select") + } + return &logStaticIterator{data: data}, nil +} diff --git a/internal/logql/logqlengine/engine.go b/internal/logql/logqlengine/engine.go index cfacf9f5..d6fdfa72 100644 --- a/internal/logql/logqlengine/engine.go +++ b/internal/logql/logqlengine/engine.go @@ -21,7 +21,7 @@ import ( // Engine is a LogQL evaluation engine. type Engine struct { querier Querier - querierCaps QuerierСapabilities + querierCaps QuerierCapabilities lookbackDuration time.Duration parseOpts logql.ParseOptions @@ -58,7 +58,7 @@ func NewEngine(querier Querier, opts Options) *Engine { return &Engine{ querier: querier, - querierCaps: querier.Сapabilities(), + querierCaps: querier.Capabilities(), lookbackDuration: opts.LookbackDuration, parseOpts: opts.ParseOptions, tracer: opts.TracerProvider.Tracer("logql.Engine"), diff --git a/internal/logql/logqlengine/engine_test.go b/internal/logql/logqlengine/engine_test.go index bd017a6d..b0aa2ebf 100644 --- a/internal/logql/logqlengine/engine_test.go +++ b/internal/logql/logqlengine/engine_test.go @@ -35,7 +35,7 @@ type mockQuerier struct { step time.Duration } -func (m *mockQuerier) Сapabilities() (caps QuerierСapabilities) { +func (m *mockQuerier) Capabilities() (caps QuerierCapabilities) { return caps } diff --git a/internal/logql/logqlengine/precondition.go b/internal/logql/logqlengine/precondition.go index bb8a8489..7a158004 100644 --- a/internal/logql/logqlengine/precondition.go +++ b/internal/logql/logqlengine/precondition.go @@ -9,7 +9,7 @@ type queryConditions struct { params SelectLogsParams } -func extractQueryConditions(caps QuerierСapabilities, sel logql.Selector, stages []logql.PipelineStage) (cond queryConditions, _ error) { +func extractQueryConditions(caps QuerierCapabilities, sel logql.Selector, stages []logql.PipelineStage) (cond queryConditions, _ error) { var prefilters []Processor for _, lm := range sel.Matchers { diff --git a/internal/logql/logqlengine/precondition_test.go b/internal/logql/logqlengine/precondition_test.go index fa02b153..2b395bda 100644 --- a/internal/logql/logqlengine/precondition_test.go +++ b/internal/logql/logqlengine/precondition_test.go @@ -67,7 +67,7 @@ func TestExtractLabelQueryConditions(t *testing.T) { for i, tt := range tests { tt := tt t.Run(fmt.Sprintf("Test%d", i+1), func(t *testing.T) { - var caps QuerierСapabilities + var caps QuerierCapabilities caps.Label.Add(tt.labelCaps...) conds, err := extractQueryConditions(caps, tt.sel, nil) @@ -142,7 +142,7 @@ func TestExtractLineQueryConditions(t *testing.T) { for i, tt := range tests { tt := tt t.Run(fmt.Sprintf("Test%d", i+1), func(t *testing.T) { - var caps QuerierСapabilities + var caps QuerierCapabilities caps.Line.Add(tt.lineCaps...) conds, err := extractQueryConditions(caps, logql.Selector{}, tt.stages) diff --git a/internal/logql/logqlengine/storage.go b/internal/logql/logqlengine/storage.go index 84c67b46..aa5be718 100644 --- a/internal/logql/logqlengine/storage.go +++ b/internal/logql/logqlengine/storage.go @@ -25,19 +25,19 @@ func (caps SupportedOps) Supports(op logql.BinOp) bool { return caps&mask != 0 } -// QuerierСapabilities defines what operations storage can do. -type QuerierСapabilities struct { +// QuerierCapabilities defines what operations storage can do. +type QuerierCapabilities struct { Label SupportedOps Line SupportedOps } // Querier does queries to storage. type Querier interface { - // Сapabilities returns Querier capabilities. + // Capabilities returns Querier capabilities. // // NOTE: engine would call once and then save value. - // Сapabilities should not change over time. - Сapabilities() QuerierСapabilities + // Capabilities should not change over time. + Capabilities() QuerierCapabilities // SelectLogs selects log records from storage. SelectLogs(ctx context.Context, start, end otelstorage.Timestamp, params SelectLogsParams) (iterators.Iterator[logstorage.Record], error) } diff --git a/internal/ytstorage/yql_querier_logs.go b/internal/ytstorage/yql_querier_logs.go index f2e125db..e53e7c45 100644 --- a/internal/ytstorage/yql_querier_logs.go +++ b/internal/ytstorage/yql_querier_logs.go @@ -19,8 +19,8 @@ import ( var _ logqlengine.Querier = (*YQLQuerier)(nil) -// Сapabilities defines storage capabilities. -func (q *YQLQuerier) Сapabilities() (caps logqlengine.QuerierСapabilities) { +// Capabilities defines storage capabilities. +func (q *YQLQuerier) Capabilities() (caps logqlengine.QuerierCapabilities) { caps.Label.Add(logql.OpEq, logql.OpNotEq, logql.OpRe, logql.OpNotRe) caps.Line.Add(logql.OpEq, logql.OpNotEq, logql.OpRe, logql.OpNotRe) return caps diff --git a/internal/ytstorage/ytql_querier_logs.go b/internal/ytstorage/ytql_querier_logs.go index 2903da85..ccd5897d 100644 --- a/internal/ytstorage/ytql_querier_logs.go +++ b/internal/ytstorage/ytql_querier_logs.go @@ -77,8 +77,8 @@ func (q *YTQLQuerier) LabelValues(ctx context.Context, labelName string, opts lo var _ logqlengine.Querier = (*YTQLQuerier)(nil) -// Сapabilities defines storage capabilities. -func (q *YTQLQuerier) Сapabilities() (caps logqlengine.QuerierСapabilities) { +// Capabilities defines storage capabilities. +func (q *YTQLQuerier) Capabilities() (caps logqlengine.QuerierCapabilities) { // FIXME(tdakkota): we don't add OpRe and OpNotRe because YT QL query executer throws an exception // when regexp function are used. caps.Label.Add(logql.OpEq, logql.OpNotEq) From eb31b35d6a344c8e59d0ed3ae9f4c35a5da62b46 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 27 Nov 2023 17:20:30 +0300 Subject: [PATCH 064/112] test(lokie2e): change json to jsonl --- integration/lokie2e/_testdata/{logs.json => logs.jsonl} | 0 integration/lokie2e/common_test.go | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename integration/lokie2e/_testdata/{logs.json => logs.jsonl} (100%) diff --git a/integration/lokie2e/_testdata/logs.json b/integration/lokie2e/_testdata/logs.jsonl similarity index 100% rename from integration/lokie2e/_testdata/logs.json rename to integration/lokie2e/_testdata/logs.jsonl diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index 0da1a890..7ec28816 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -70,7 +70,7 @@ func runTest( querier logstorage.Querier, engineQuerier logqlengine.Querier, ) { - set, err := readBatchSet("_testdata/logs.json") + set, err := readBatchSet("_testdata/logs.jsonl") require.NoError(t, err) require.NotEmpty(t, set.Batches) require.NotEmpty(t, set.Labels) From d2d4b7d52db1449e67369da4a4eb42c9f2b44424 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 27 Nov 2023 17:31:04 +0300 Subject: [PATCH 065/112] test(lokie2e): always send time range --- integration/lokie2e/common_test.go | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index 7ec28816..c54735a0 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -83,7 +83,11 @@ func runTest( t.Run("Labels", func(t *testing.T) { a := require.New(t) - r, err := c.Labels(ctx, lokiapi.LabelsParams{}) + r, err := c.Labels(ctx, lokiapi.LabelsParams{ + // Always sending time range because default is current time. + Start: lokiapi.NewOptLokiTime(asLokiTime(set.Start)), + End: lokiapi.NewOptLokiTime(asLokiTime(set.End)), + }) a.NoError(err) a.Len(r.Data, len(set.Labels)) for _, label := range r.Data { @@ -99,7 +103,12 @@ func runTest( labelValue[t.Value] = struct{}{} } - r, err := c.LabelValues(ctx, lokiapi.LabelValuesParams{Name: labelName}) + r, err := c.LabelValues(ctx, lokiapi.LabelValuesParams{ + Name: labelName, + // Always sending time range because default is current time. + Start: lokiapi.NewOptLokiTime(asLokiTime(set.Start)), + End: lokiapi.NewOptLokiTime(asLokiTime(set.End)), + }) a.NoError(err) a.Len(r.Data, len(labelValue)) for _, val := range r.Data { @@ -206,6 +215,7 @@ func runTest( resp, err := c.QueryRange(ctx, lokiapi.QueryRangeParams{ Query: tt.query, + // Always sending time range because default is current time. Start: lokiapi.NewOptLokiTime(asLokiTime(set.Start)), End: lokiapi.NewOptLokiTime(asLokiTime(set.End)), Limit: lokiapi.NewOptInt(1000), From a407f860516fd300e368d06aefad4ee24520cb7d Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 27 Nov 2023 17:32:25 +0300 Subject: [PATCH 066/112] fix(chstorage.querier.logs): fix iterator --- internal/chstorage/querier_logs.go | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index 7c80e153..3eacccaa 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -41,7 +41,6 @@ func (q *Querier) LabelNames(ctx context.Context, opts logstorage.LabelsOptions) } span.End() }() - var ( names proto.ColStr out []string @@ -60,7 +59,8 @@ func (q *Querier) LabelNames(ctx context.Context, opts logstorage.LabelsOptions) Body: fmt.Sprintf(`SELECT DISTINCT arrayJoin(arrayConcat(JSONExtractKeys(attributes), JSONExtractKeys(resource), JSONExtractKeys(scope_attributes))) as key FROM %s -WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d)`, +WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d) +LIMIT 1000`, table, opts.Start, opts.End, ), }); err != nil { @@ -101,6 +101,7 @@ func (l *labelStaticIterator) Next(t *logstorage.Label) bool { t.Type = int32(pcommon.ValueTypeStr) t.Value = l.values[0].String() } + l.values = l.values[1:] return true } @@ -124,7 +125,6 @@ func (q *Querier) LabelValues(ctx context.Context, labelName string, opts logsto } span.End() }() - var ( names proto.ColStr out []jx.Raw @@ -147,13 +147,12 @@ COALESCE( JSONExtractRaw(resource, %[1]s) ) as value FROM %s -WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d)`, +WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d) LIMIT 1000`, singleQuoted(labelName), table, opts.Start, opts.End, ), }); err != nil { return nil, errors.Wrap(err, "select") } - return &labelStaticIterator{ name: labelName, values: out, @@ -250,6 +249,7 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta // TODO: use streaming. var data []logstorage.Record + if err := q.ch.Do(ctx, ch.Query{ Logger: zctx.From(ctx).Named("ch"), Body: query.String(), @@ -260,7 +260,6 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta }); err != nil { return errors.Wrap(err, "for each") } - out.Reset() return nil }, }); err != nil { From bb7931cac0e936f6e3cac9df78ffded1e87d7c95 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Mon, 27 Nov 2023 17:39:20 +0300 Subject: [PATCH 067/112] feat(chstorage.querier.logs): support REGEXP --- internal/chstorage/querier_logs.go | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index 3eacccaa..cbd1ef8e 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -213,9 +213,9 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta fmt.Fprintf(&query, " FROM %s WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d)", table, start, end) for _, m := range params.Labels { switch m.Op { - case logql.OpEq: + case logql.OpEq, logql.OpRe: query.WriteString(" AND (") - case logql.OpNotEq: + case logql.OpNotEq, logql.OpNotRe: query.WriteString(" AND NOT (") default: return nil, errors.Errorf("unexpected op %q", m.Op) @@ -229,22 +229,31 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta query.WriteString(" OR ") } // TODO: how to match integers, booleans, floats, arrays? - fmt.Fprintf(&query, "JSONExtractString(%s, %s) = %s", column, singleQuoted(m.Label), singleQuoted(m.Value)) + switch m.Op { + case logql.OpEq, logql.OpNotEq: + fmt.Fprintf(&query, "JSONExtractString(%s, %s) = %s", column, singleQuoted(m.Label), singleQuoted(m.Value)) + case logql.OpRe, logql.OpNotRe: + fmt.Fprintf(&query, "JSONExtractString(%s, %s) REGEXP %s", column, singleQuoted(m.Label), singleQuoted(m.Value)) + } } query.WriteByte(')') } for _, m := range params.Line { switch m.Op { - case logql.OpEq: + case logql.OpEq, logql.OpRe: query.WriteString(" AND ") - case logql.OpNotEq: + case logql.OpNotEq, logql.OpNotRe: query.WriteString(" AND NOT ") default: return nil, errors.Errorf("unexpected op %q", m.Op) } - // Line filter checks if line contains given value. - fmt.Fprintf(&query, "positionUTF8(body, %s) > 0", singleQuoted(m.Value)) + switch m.Op { + case logql.OpEq, logql.OpNotEq: + fmt.Fprintf(&query, "positionUTF8(body, %s) > 0", singleQuoted(m.Value)) + case logql.OpRe, logql.OpNotRe: + fmt.Fprintf(&query, "body REGEXP %s", singleQuoted(m.Value)) + } } // TODO: use streaming. From 085a8bf97f7dab066fa59ef6d6ee1fa3d18af51d Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 28 Nov 2023 02:44:56 +0300 Subject: [PATCH 068/112] feat(chstorage): save metric flags too --- internal/chstorage/columns_metrics.go | 12 ++++++++++++ internal/chstorage/inserter_metrics.go | 8 ++++++++ internal/chstorage/schema_metrics.go | 4 ++++ 3 files changed, 24 insertions(+) diff --git a/internal/chstorage/columns_metrics.go b/internal/chstorage/columns_metrics.go index eca23d63..9396b6f8 100644 --- a/internal/chstorage/columns_metrics.go +++ b/internal/chstorage/columns_metrics.go @@ -10,6 +10,7 @@ type pointColumns struct { value proto.ColFloat64 + flags proto.ColUInt32 attributes proto.ColStr resource proto.ColStr } @@ -28,6 +29,7 @@ func (c *pointColumns) Input() proto.Input { {Name: "value", Data: c.value}, + {Name: "flags", Data: c.flags}, {Name: "attributes", Data: c.attributes}, {Name: "resource", Data: c.resource}, } @@ -41,6 +43,7 @@ func (c *pointColumns) Result() proto.Results { {Name: "value", Data: &c.value}, + {Name: "flags", Data: &c.flags}, {Name: "attributes", Data: &c.attributes}, {Name: "resource", Data: &c.resource}, } @@ -57,6 +60,7 @@ type histogramColumns struct { bucketCounts *proto.ColArr[uint64] explicitBounds *proto.ColArr[float64] + flags proto.ColUInt32 attributes proto.ColStr resource proto.ColStr } @@ -86,6 +90,7 @@ func (c *histogramColumns) Input() proto.Input { {Name: "histogram_bucket_counts", Data: c.bucketCounts}, {Name: "histogram_explicit_bounds", Data: c.explicitBounds}, + {Name: "flags", Data: c.flags}, {Name: "attributes", Data: c.attributes}, {Name: "resource", Data: c.resource}, } @@ -104,6 +109,7 @@ func (c *histogramColumns) Result() proto.Results { {Name: "histogram_bucket_counts", Data: c.bucketCounts}, {Name: "histogram_explicit_bounds", Data: c.explicitBounds}, + {Name: "flags", Data: &c.flags}, {Name: "attributes", Data: &c.attributes}, {Name: "resource", Data: &c.resource}, } @@ -124,6 +130,7 @@ type expHistogramColumns struct { negativeOffset proto.ColInt32 negativeBucketCounts *proto.ColArr[uint64] + flags proto.ColUInt32 attributes proto.ColStr resource proto.ColStr } @@ -157,6 +164,7 @@ func (c *expHistogramColumns) Input() proto.Input { {Name: "exp_histogram_negative_offset", Data: c.negativeOffset}, {Name: "exp_histogram_negative_bucket_counts", Data: c.negativeBucketCounts}, + {Name: "flags", Data: c.flags}, {Name: "attributes", Data: c.attributes}, {Name: "resource", Data: c.resource}, } @@ -179,6 +187,7 @@ func (c *expHistogramColumns) Result() proto.Results { {Name: "exp_histogram_negative_offset", Data: &c.negativeOffset}, {Name: "exp_histogram_negative_bucket_counts", Data: c.negativeBucketCounts}, + {Name: "flags", Data: &c.flags}, {Name: "attributes", Data: &c.attributes}, {Name: "resource", Data: &c.resource}, } @@ -193,6 +202,7 @@ type summaryColumns struct { quantiles *proto.ColArr[float64] values *proto.ColArr[float64] + flags proto.ColUInt32 attributes proto.ColStr resource proto.ColStr } @@ -217,6 +227,7 @@ func (c *summaryColumns) Input() proto.Input { {Name: "summary_quantiles", Data: c.quantiles}, {Name: "summary_values", Data: c.values}, + {Name: "flags", Data: c.flags}, {Name: "attributes", Data: c.attributes}, {Name: "resource", Data: c.resource}, } @@ -233,6 +244,7 @@ func (c *summaryColumns) Result() proto.Results { {Name: "summary_quantiles", Data: c.quantiles}, {Name: "summary_values", Data: c.values}, + {Name: "flags", Data: &c.flags}, {Name: "attributes", Data: &c.attributes}, {Name: "resource", Data: &c.resource}, } diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index 491f5f7e..fa4e142b 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -85,6 +85,7 @@ func (b *metricsBatch) addPoints(name string, res pcommon.Map, slice pmetric.Num for i := 0; i < slice.Len(); i++ { point := slice.At(i) ts := point.Timestamp().AsTime() + flags := point.Flags() attrs := point.Attributes() var val float64 @@ -105,6 +106,7 @@ func (b *metricsBatch) addPoints(name string, res pcommon.Map, slice pmetric.Num c.name.Append(name) c.timestamp.Append(ts) c.value.Append(val) + c.flags.Append(uint32(flags)) c.attributes.Append(encodeAttributes(attrs)) c.resource.Append(encodeAttributes(res)) } @@ -116,6 +118,7 @@ func (b *metricsBatch) addHistogramPoints(name string, res pcommon.Map, slice pm for i := 0; i < slice.Len(); i++ { point := slice.At(i) ts := point.Timestamp().AsTime() + flags := point.Flags() attrs := point.Attributes() count := point.Count() sum := proto.Nullable[float64]{ @@ -142,6 +145,7 @@ func (b *metricsBatch) addHistogramPoints(name string, res pcommon.Map, slice pm c.max.Append(max) c.bucketCounts.Append(bucketCounts) c.explicitBounds.Append(explicitBounds) + c.flags.Append(uint32(flags)) c.attributes.Append(encodeAttributes(attrs)) c.resource.Append(encodeAttributes(res)) } @@ -160,6 +164,7 @@ func (b *metricsBatch) addExpHistogramPoints(name string, res pcommon.Map, slice for i := 0; i < slice.Len(); i++ { point := slice.At(i) ts := point.Timestamp().AsTime() + flags := point.Flags() attrs := point.Attributes() count := point.Count() sum := proto.Nullable[float64]{ @@ -193,6 +198,7 @@ func (b *metricsBatch) addExpHistogramPoints(name string, res pcommon.Map, slice c.positiveBucketCounts.Append(positiveBucketCounts) c.negativeOffset.Append(negativeOffset) c.negativeBucketCounts.Append(negativeBucketCounts) + c.flags.Append(uint32(flags)) c.attributes.Append(encodeAttributes(attrs)) c.resource.Append(encodeAttributes(res)) } @@ -204,6 +210,7 @@ func (b *metricsBatch) addSummaryPoints(name string, res pcommon.Map, slice pmet for i := 0; i < slice.Len(); i++ { point := slice.At(i) ts := point.Timestamp().AsTime() + flags := point.Flags() attrs := point.Attributes() count := point.Count() sum := point.Sum() @@ -227,6 +234,7 @@ func (b *metricsBatch) addSummaryPoints(name string, res pcommon.Map, slice pmet c.sum.Append(sum) c.quantiles.Append(quantiles) c.values.Append(values) + c.flags.Append(uint32(flags)) c.attributes.Append(encodeAttributes(attrs)) c.resource.Append(encodeAttributes(res)) } diff --git a/internal/chstorage/schema_metrics.go b/internal/chstorage/schema_metrics.go index 9742ba16..5a7fb590 100644 --- a/internal/chstorage/schema_metrics.go +++ b/internal/chstorage/schema_metrics.go @@ -15,6 +15,7 @@ const ( value Float64, + flags UInt32, attributes String, resource String ) @@ -32,6 +33,7 @@ const ( histogram_bucket_counts Array(UInt64), histogram_explicit_bounds Array(Float64), + flags UInt32, attributes String, resource String ) @@ -53,6 +55,7 @@ const ( exp_histogram_negative_offset Int32, exp_histogram_negative_bucket_counts Array(UInt64), + flags UInt32, attributes String, resource String ) @@ -68,6 +71,7 @@ const ( summary_quantiles Array(Float64), summary_values Array(Float64), + flags UInt32, attributes String, resource String ) From 1ef66da0b5c55663981905ed430e7e1c0b42112c Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 28 Nov 2023 03:30:19 +0300 Subject: [PATCH 069/112] feat(chstorage): query exponential histograms too --- internal/chstorage/querier_metrics.go | 349 +++++++++++--------- internal/chstorage/querier_metrics_hist.go | 246 ++++++++++++++ internal/chstorage/querier_metrics_point.go | 109 ++++++ 3 files changed, 552 insertions(+), 152 deletions(-) create mode 100644 internal/chstorage/querier_metrics_hist.go create mode 100644 internal/chstorage/querier_metrics_point.go diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 819ee3fc..e453f77c 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -3,20 +3,19 @@ package chstorage import ( "context" "fmt" - "slices" "strings" "time" "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/proto" "github.com/go-faster/errors" - "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/annotations" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" + "golang.org/x/sync/errgroup" ) var _ storage.Queryable = (*Querier)(nil) @@ -187,7 +186,6 @@ type seriesKey struct { } func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHints, matchers ...*labels.Matcher) (_ storage.SeriesSet, rerr error) { - table := p.tables.Points var ( start = p.mint end = p.maxt @@ -205,7 +203,6 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin trace.WithAttributes( attribute.Int64("chstorage.start_range", start.UnixNano()), attribute.Int64("chstorage.end_range", end.UnixNano()), - attribute.String("chstorage.table", table), ), ) defer func() { @@ -215,61 +212,104 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin span.End() }() - var query strings.Builder - fmt.Fprintf(&query, "SELECT * FROM %#[1]q WHERE true\n", table) - if !start.IsZero() { - fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(timestamp) >= %d\n", start.UnixNano()) - } - if !end.IsZero() { - fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(timestamp) <= %d\n", end.UnixNano()) - } - for _, m := range matchers { - switch m.Type { - case labels.MatchEqual, labels.MatchRegexp: - query.WriteString("AND ") - case labels.MatchNotEqual, labels.MatchNotRegexp: - query.WriteString("AND NOT ") - default: - return nil, errors.Errorf("unexpected type %q", m.Type) + buildQuery := func(table string) (string, error) { + var query strings.Builder + fmt.Fprintf(&query, "SELECT * FROM %#[1]q WHERE true\n", table) + if !start.IsZero() { + fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(timestamp) >= %d\n", start.UnixNano()) } - - { - selectors := []string{ - "name", + if !end.IsZero() { + fmt.Fprintf(&query, "\tAND toUnixTimestamp64Nano(timestamp) <= %d\n", end.UnixNano()) + } + for _, m := range matchers { + switch m.Type { + case labels.MatchEqual, labels.MatchRegexp: + query.WriteString("AND ") + case labels.MatchNotEqual, labels.MatchNotRegexp: + query.WriteString("AND NOT ") + default: + return "", errors.Errorf("unexpected type %q", m.Type) } - if m.Name != "__name__" { - selectors = []string{ - fmt.Sprintf("JSONExtractString(attributes, %s)", singleQuoted(m.Name)), - fmt.Sprintf("JSONExtractString(resource, %s)", singleQuoted(m.Name)), + + { + selectors := []string{ + "name", } - } - query.WriteString("(\n") - for i, sel := range selectors { - if i != 0 { - query.WriteString("\tOR ") + if m.Name != "__name__" { + selectors = []string{ + fmt.Sprintf("JSONExtractString(attributes, %s)", singleQuoted(m.Name)), + fmt.Sprintf("JSONExtractString(resource, %s)", singleQuoted(m.Name)), + } } - // Note: predicate negated above. - switch m.Type { - case labels.MatchEqual, labels.MatchNotEqual: - fmt.Fprintf(&query, "%s = %s\n", sel, singleQuoted(m.Value)) - case labels.MatchRegexp, labels.MatchNotRegexp: - fmt.Fprintf(&query, "%s REGEXP %s\n", sel, singleQuoted(m.Value)) - default: - return nil, errors.Errorf("unexpected type %q", m.Type) + query.WriteString("(\n") + for i, sel := range selectors { + if i != 0 { + query.WriteString("\tOR ") + } + // Note: predicate negated above. + switch m.Type { + case labels.MatchEqual, labels.MatchNotEqual: + fmt.Fprintf(&query, "%s = %s\n", sel, singleQuoted(m.Value)) + case labels.MatchRegexp, labels.MatchNotRegexp: + fmt.Fprintf(&query, "%s REGEXP %s\n", sel, singleQuoted(m.Value)) + default: + return "", errors.Errorf("unexpected type %q", m.Type) + } } + query.WriteString(")") } - query.WriteString(")") + query.WriteString("\n") + } + query.WriteString("ORDER BY timestamp") + return query.String(), nil + } + + var ( + points []storage.Series + histSeries []storage.Series + ) + grp, grpCtx := errgroup.WithContext(ctx) + grp.Go(func() error { + ctx := grpCtx + + query, err := buildQuery(p.tables.Points) + if err != nil { + return err + } + + result, err := p.queryPoints(ctx, query) + if err != nil { + return errors.Wrap(err, "query points") + } + points = result + return nil + }) + grp.Go(func() error { + ctx := grpCtx + + query, err := buildQuery(p.tables.ExpHistograms) + if err != nil { + return err + } + + result, err := p.queryExpHistrograms(ctx, query) + if err != nil { + return errors.Wrap(err, "query histrograms") } - query.WriteString("\n") + histSeries = result + return nil + }) + if err := grp.Wait(); err != nil { + return nil, err } - query.WriteString("ORDER BY timestamp") - return p.doQuery(ctx, query.String()) + points = append(points, histSeries...) + return newSeriesSet(points), nil } -func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.SeriesSet, error) { +func (p *promQuerier) queryPoints(ctx context.Context, query string) ([]storage.Series, error) { type seriesWithLabels struct { - series *series + series *series[pointData] labels map[string]string } @@ -296,13 +336,100 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series s, ok := set[key] if !ok { s = seriesWithLabels{ - series: &series{}, + series: &series[pointData]{}, + labels: map[string]string{}, + } + set[key] = s + } + + s.series.data.values = append(s.series.data.values, value) + s.series.ts = append(s.series.ts, timestamp.UnixMilli()) + + s.labels["__name__"] = name + if err := parseLabels(resource, s.labels); err != nil { + return errors.Wrap(err, "parse resource") + } + if err := parseLabels(attributes, s.labels); err != nil { + return errors.Wrap(err, "parse attributes") + } + } + return nil + }, + }); err != nil { + return nil, errors.Wrap(err, "do query") + } + + var ( + result = make([]storage.Series, 0, len(set)) + lb labels.ScratchBuilder + ) + for _, s := range set { + lb.Reset() + for key, value := range s.labels { + lb.Add(key, value) + } + lb.Sort() + s.series.labels = lb.Labels() + result = append(result, s.series) + } + + return result, nil +} + +func (p *promQuerier) queryExpHistrograms(ctx context.Context, query string) ([]storage.Series, error) { + type seriesWithLabels struct { + series *series[expHistData] + labels map[string]string + } + + var ( + set = map[seriesKey]seriesWithLabels{} + c = newExpHistogramColumns() + ) + if err := p.ch.Do(ctx, ch.Query{ + Body: query, + Result: c.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + for i := 0; i < c.timestamp.Rows(); i++ { + name := c.name.Row(i) + timestamp := c.timestamp.Row(i) + count := c.count.Row(i) + sum := c.sum.Row(i) + min := c.min.Row(i) + max := c.max.Row(i) + scale := c.scale.Row(i) + zerocount := c.zerocount.Row(i) + positiveOffset := c.positiveOffset.Row(i) + positiveBucketCounts := c.positiveBucketCounts.Row(i) + negativeOffset := c.negativeOffset.Row(i) + negativeBucketCounts := c.negativeBucketCounts.Row(i) + attributes := c.attributes.Row(i) + resource := c.resource.Row(i) + + key := seriesKey{ + name: name, + attributes: attributes, + resource: resource, + } + s, ok := set[key] + if !ok { + s = seriesWithLabels{ + series: &series[expHistData]{}, labels: map[string]string{}, } set[key] = s } - s.series.values = append(s.series.values, value) + s.series.data.count = append(s.series.data.count, count) + s.series.data.sum = append(s.series.data.sum, sum) + s.series.data.min = append(s.series.data.min, min) + s.series.data.max = append(s.series.data.max, max) + s.series.data.scale = append(s.series.data.scale, scale) + s.series.data.zerocount = append(s.series.data.zerocount, zerocount) + s.series.data.positiveOffset = append(s.series.data.positiveOffset, positiveOffset) + s.series.data.positiveBucketCounts = append(s.series.data.positiveBucketCounts, positiveBucketCounts) + s.series.data.negativeOffset = append(s.series.data.negativeOffset, negativeOffset) + s.series.data.negativeBucketCounts = append(s.series.data.negativeBucketCounts, negativeBucketCounts) s.series.ts = append(s.series.ts, timestamp.UnixMilli()) s.labels["__name__"] = name @@ -320,7 +447,7 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series } var ( - result = make([]*series, 0, len(set)) + result = make([]storage.Series, 0, len(set)) lb labels.ScratchBuilder ) for _, s := range set { @@ -333,24 +460,24 @@ func (p *promQuerier) doQuery(ctx context.Context, query string) (storage.Series result = append(result, s.series) } - return newSeriesSet(result), nil + return result, nil } -type seriesSet struct { - set []*series +type seriesSet[S storage.Series] struct { + set []S n int } -func newSeriesSet(set []*series) *seriesSet { - return &seriesSet{ +func newSeriesSet[S storage.Series](set []S) *seriesSet[S] { + return &seriesSet[S]{ set: set, n: -1, } } -var _ storage.SeriesSet = (*seriesSet)(nil) +var _ storage.SeriesSet = (*seriesSet[storage.Series])(nil) -func (s *seriesSet) Next() bool { +func (s *seriesSet[S]) Next() bool { if s.n+1 >= len(s.set) { return false } @@ -359,32 +486,36 @@ func (s *seriesSet) Next() bool { } // At returns full series. Returned series should be iterable even after Next is called. -func (s *seriesSet) At() storage.Series { +func (s *seriesSet[S]) At() storage.Series { return s.set[s.n] } // The error that iteration as failed with. // When an error occurs, set cannot continue to iterate. -func (s *seriesSet) Err() error { +func (s *seriesSet[S]) Err() error { return nil } // A collection of warnings for the whole set. // Warnings could be return even iteration has not failed with error. -func (s *seriesSet) Warnings() annotations.Annotations { +func (s *seriesSet[S]) Warnings() annotations.Annotations { return nil } -type series struct { +type seriesData interface { + Iterator(ts []int64) chunkenc.Iterator +} + +type series[Data seriesData] struct { labels labels.Labels - values []float64 + data Data ts []int64 } -var _ storage.Series = (*series)(nil) +var _ storage.Series = (*series[pointData])(nil) // Labels returns the complete set of labels. For series it means all labels identifying the series. -func (s *series) Labels() labels.Labels { +func (s *series[Data]) Labels() labels.Labels { return s.labels } @@ -392,92 +523,6 @@ func (s *series) Labels() labels.Labels { // The iterator passed as argument is for re-use, if not nil. // Depending on implementation, the iterator can // be re-used or a new iterator can be allocated. -func (s *series) Iterator(chunkenc.Iterator) chunkenc.Iterator { - return newPointIterator(s.values, s.ts) -} - -type pointIterator struct { - values []float64 - ts []int64 - n int -} - -var _ chunkenc.Iterator = (*pointIterator)(nil) - -func newPointIterator(values []float64, ts []int64) *pointIterator { - return &pointIterator{ - values: values, - ts: ts, - n: -1, - } -} - -// Next advances the iterator by one and returns the type of the value -// at the new position (or ValNone if the iterator is exhausted). -func (p *pointIterator) Next() chunkenc.ValueType { - if p.n+1 >= len(p.values) { - return chunkenc.ValNone - } - p.n++ - return chunkenc.ValFloat -} - -// Seek advances the iterator forward to the first sample with a -// timestamp equal or greater than t. If the current sample found by a -// previous `Next` or `Seek` operation already has this property, Seek -// has no effect. If a sample has been found, Seek returns the type of -// its value. Otherwise, it returns ValNone, after which the iterator is -// exhausted. -func (p *pointIterator) Seek(seek int64) chunkenc.ValueType { - // Find the closest value. - idx, _ := slices.BinarySearch(p.ts, seek) - switch { - case idx >= len(p.ts): - // Outside of the range. - p.n = len(p.ts) - return chunkenc.ValNone - case idx < 1: - // Move to the first point. - p.n = 0 - return chunkenc.ValFloat - default: - p.n = idx - 1 - return chunkenc.ValFloat - } -} - -// At returns the current timestamp/value pair if the value is a float. -// Before the iterator has advanced, the behavior is unspecified. -func (p *pointIterator) At() (t int64, v float64) { - t = p.AtT() - v = p.values[p.n] - return t, v -} - -// AtHistogram returns the current timestamp/value pair if the value is -// a histogram with integer counts. Before the iterator has advanced, -// the behavior is unspecified. -func (p *pointIterator) AtHistogram() (int64, *histogram.Histogram) { - return 0, nil -} - -// AtFloatHistogram returns the current timestamp/value pair if the -// value is a histogram with floating-point counts. It also works if the -// value is a histogram with integer counts, in which case a -// FloatHistogram copy of the histogram is returned. Before the iterator -// has advanced, the behavior is unspecified. -func (p *pointIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) { - return 0, nil -} - -// AtT returns the current timestamp. -// Before the iterator has advanced, the behavior is unspecified. -func (p *pointIterator) AtT() int64 { - return p.ts[p.n] -} - -// Err returns the current error. It should be used only after the -// iterator is exhausted, i.e. `Next` or `Seek` have returned ValNone. -func (p *pointIterator) Err() error { - return nil +func (s *series[Data]) Iterator(chunkenc.Iterator) chunkenc.Iterator { + return s.data.Iterator(s.ts) } diff --git a/internal/chstorage/querier_metrics_hist.go b/internal/chstorage/querier_metrics_hist.go new file mode 100644 index 00000000..a6c42c6b --- /dev/null +++ b/internal/chstorage/querier_metrics_hist.go @@ -0,0 +1,246 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package chstorage + +import ( + "math" + + "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/errors" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/value" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +type expHistData struct { + count []uint64 + sum []proto.Nullable[float64] + min []proto.Nullable[float64] + max []proto.Nullable[float64] + scale []int32 + zerocount []uint64 + positiveOffset []int32 + positiveBucketCounts [][]uint64 + negativeOffset []int32 + negativeBucketCounts [][]uint64 + flags []uint32 +} + +func (e expHistData) Iterator(ts []int64) chunkenc.Iterator { + return newExpHistIterator(e, ts) +} + +type expHistIterator struct { + values expHistData + + current histogram.Histogram + err error + + ts []int64 + n int +} + +var _ chunkenc.Iterator = (*expHistIterator)(nil) + +func newExpHistIterator(values expHistData, ts []int64) *expHistIterator { + return &expHistIterator{ + values: values, + ts: ts, + n: -1, + } +} + +// Next advances the iterator by one and returns the type of the value +// at the new position (or ValNone if the iterator is exhausted). +func (p *expHistIterator) Next() chunkenc.ValueType { + if p.n+1 >= len(p.ts) { + return chunkenc.ValNone + } + p.n++ + p.current, p.err = p.loadValue() + return chunkenc.ValFloatHistogram +} + +// Seek advances the iterator forward to the first sample with a +// timestamp equal or greater than t. If the current sample found by a +// previous `Next` or `Seek` operation already has this property, Seek +// has no effect. If a sample has been found, Seek returns the type of +// its value. Otherwise, it returns ValNone, after which the iterator is +// exhausted. +func (p *expHistIterator) Seek(seek int64) chunkenc.ValueType { + // Find the closest value. + if !seekIterator(p.ts, &p.n, seek) { + return chunkenc.ValNone + } + p.current, p.err = p.loadValue() + return chunkenc.ValFloatHistogram +} + +const defaultZeroThreshold = 1e-128 + +func (p *expHistIterator) loadValue() (h histogram.Histogram, _ error) { + scale := p.values.scale[p.n] + if scale < -4 { + return h, errors.Errorf("cannot convert histogram, scale must be >= -4, was %d", scale) + } + + var scaleDown int32 + if scale > 8 { + scaleDown = scale - 8 + scale = 8 + } + + pSpans, pDeltas := convertBucketsLayout(p.values.positiveOffset[p.n], p.values.positiveBucketCounts[p.n], scaleDown) + nSpans, nDeltas := convertBucketsLayout(p.values.negativeOffset[p.n], p.values.negativeBucketCounts[p.n], scaleDown) + + h = histogram.Histogram{ + Schema: scale, + + ZeroCount: p.values.zerocount[p.n], + // TODO use zero_threshold, if set, see + // https://github.com/open-telemetry/opentelemetry-proto/pull/441 + ZeroThreshold: defaultZeroThreshold, + + PositiveSpans: pSpans, + PositiveBuckets: pDeltas, + NegativeSpans: nSpans, + NegativeBuckets: nDeltas, + } + + if flags := pmetric.DataPointFlags(p.values.flags[p.n]); flags.NoRecordedValue() { + h.Sum = math.Float64frombits(value.StaleNaN) + h.Count = value.StaleNaN + } else { + if sum := p.values.sum[p.n]; sum.Set { + h.Sum = sum.Value + } + h.Count = p.values.count[p.n] + } + return h, nil +} + +// convertBucketsLayout translates OTel Exponential Histogram dense buckets +// representation to Prometheus Native Histogram sparse bucket representation. +// +// The translation logic is taken from the client_golang `histogram.go#makeBuckets` +// function, see `makeBuckets` https://github.com/prometheus/client_golang/blob/main/prometheus/histogram.go +// The bucket indexes conversion was adjusted, since OTel exp. histogram bucket +// index 0 corresponds to the range (1, base] while Prometheus bucket index 0 +// to the range (base 1]. +// +// scaleDown is the factor by which the buckets are scaled down. In other words 2^scaleDown buckets will be merged into one. +func convertBucketsLayout(offset int32, bucketCounts []uint64, scaleDown int32) (spans []histogram.Span, deltas []int64) { + numBuckets := len(bucketCounts) + if numBuckets == 0 { + return nil, nil + } + + var ( + count int64 + prevCount int64 + ) + + appendDelta := func(count int64) { + spans[len(spans)-1].Length++ + deltas = append(deltas, count-prevCount) + prevCount = count + } + + // The offset is scaled and adjusted by 1 as described above. + bucketIdx := offset>>scaleDown + 1 + spans = append(spans, histogram.Span{ + Offset: bucketIdx, + Length: 0, + }) + + for i := 0; i < numBuckets; i++ { + // The offset is scaled and adjusted by 1 as described above. + nextBucketIdx := (int32(i)+offset)>>scaleDown + 1 + if bucketIdx == nextBucketIdx { // We have not collected enough buckets to merge yet. + count += int64(bucketCounts[i]) + continue + } + if count == 0 { + count = int64(bucketCounts[i]) + continue + } + + gap := nextBucketIdx - bucketIdx - 1 + if gap > 2 { + // We have to create a new span, because we have found a gap + // of more than two buckets. The constant 2 is copied from the logic in + // https://github.com/prometheus/client_golang/blob/27f0506d6ebbb117b6b697d0552ee5be2502c5f2/prometheus/histogram.go#L1296 + spans = append(spans, histogram.Span{ + Offset: gap, + Length: 0, + }) + } else { + // We have found a small gap (or no gap at all). + // Insert empty buckets as needed. + for j := int32(0); j < gap; j++ { + appendDelta(0) + } + } + appendDelta(count) + count = int64(bucketCounts[i]) + bucketIdx = nextBucketIdx + } + // Need to use the last item's index. The offset is scaled and adjusted by 1 as described above. + gap := (int32(numBuckets)+offset-1)>>scaleDown + 1 - bucketIdx + if gap > 2 { + // We have to create a new span, because we have found a gap + // of more than two buckets. The constant 2 is copied from the logic in + // https://github.com/prometheus/client_golang/blob/27f0506d6ebbb117b6b697d0552ee5be2502c5f2/prometheus/histogram.go#L1296 + spans = append(spans, histogram.Span{ + Offset: gap, + Length: 0, + }) + } else { + // We have found a small gap (or no gap at all). + // Insert empty buckets as needed. + for j := int32(0); j < gap; j++ { + appendDelta(0) + } + } + appendDelta(count) + + return spans, deltas +} + +// At returns the current timestamp/value pair if the value is a float. +// Before the iterator has advanced, the behavior is unspecified. +func (p *expHistIterator) At() (t int64, v float64) { + return 0, 0 +} + +// AtHistogram returns the current timestamp/value pair if the value is +// a histogram with integer counts. Before the iterator has advanced, +// the behavior is unspecified. +func (p *expHistIterator) AtHistogram() (t int64, h *histogram.Histogram) { + t = p.AtT() + h = &p.current + return t, h +} + +// AtFloatHistogram returns the current timestamp/value pair if the +// value is a histogram with floating-point counts. It also works if the +// value is a histogram with integer counts, in which case a +// FloatHistogram copy of the histogram is returned. Before the iterator +// has advanced, the behavior is unspecified. +func (p *expHistIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) { + return 0, nil +} + +// AtT returns the current timestamp. +// Before the iterator has advanced, the behavior is unspecified. +func (p *expHistIterator) AtT() int64 { + return p.ts[p.n] +} + +// Err returns the current error. It should be used only after the +// iterator is exhausted, i.e. `Next` or `Seek` have returned ValNone. +func (p *expHistIterator) Err() error { + return p.err +} diff --git a/internal/chstorage/querier_metrics_point.go b/internal/chstorage/querier_metrics_point.go new file mode 100644 index 00000000..df437b76 --- /dev/null +++ b/internal/chstorage/querier_metrics_point.go @@ -0,0 +1,109 @@ +package chstorage + +import ( + "slices" + + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/tsdb/chunkenc" +) + +func seekIterator(ts []int64, n *int, seek int64) bool { + idx, _ := slices.BinarySearch(ts, seek) + switch { + case idx >= len(ts): + // Outside of the range. + *n = len(ts) + return false + case idx < 1: + // Move to the first point. + *n = 0 + return true + default: + *n = idx - 1 + return true + } +} + +type pointData struct { + values []float64 +} + +func (e pointData) Iterator(ts []int64) chunkenc.Iterator { + return newPointIterator(e.values, ts) +} + +type pointIterator struct { + values []float64 + ts []int64 + n int +} + +var _ chunkenc.Iterator = (*pointIterator)(nil) + +func newPointIterator(values []float64, ts []int64) *pointIterator { + return &pointIterator{ + values: values, + ts: ts, + n: -1, + } +} + +// Next advances the iterator by one and returns the type of the value +// at the new position (or ValNone if the iterator is exhausted). +func (p *pointIterator) Next() chunkenc.ValueType { + if p.n+1 >= len(p.ts) { + return chunkenc.ValNone + } + p.n++ + return chunkenc.ValFloat +} + +// Seek advances the iterator forward to the first sample with a +// timestamp equal or greater than t. If the current sample found by a +// previous `Next` or `Seek` operation already has this property, Seek +// has no effect. If a sample has been found, Seek returns the type of +// its value. Otherwise, it returns ValNone, after which the iterator is +// exhausted. +func (p *pointIterator) Seek(seek int64) chunkenc.ValueType { + // Find the closest value. + if !seekIterator(p.ts, &p.n, seek) { + return chunkenc.ValNone + } + return chunkenc.ValFloat +} + +// At returns the current timestamp/value pair if the value is a float. +// Before the iterator has advanced, the behavior is unspecified. +func (p *pointIterator) At() (t int64, v float64) { + t = p.AtT() + v = p.values[p.n] + return t, v +} + +// AtHistogram returns the current timestamp/value pair if the value is +// a histogram with integer counts. Before the iterator has advanced, +// the behavior is unspecified. +func (p *pointIterator) AtHistogram() (int64, *histogram.Histogram) { + return 0, nil +} + +// AtFloatHistogram returns the current timestamp/value pair if the +// value is a histogram with floating-point counts. It also works if the +// value is a histogram with integer counts, in which case a +// FloatHistogram copy of the histogram is returned. Before the iterator +// has advanced, the behavior is unspecified. +func (p *pointIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) { + return 0, nil +} + +// AtT returns the current timestamp. +// Before the iterator has advanced, the behavior is unspecified. +func (p *pointIterator) AtT() int64 { + return p.ts[p.n] +} + +// Err returns the current error. It should be used only after the +// iterator is exhausted, i.e. `Next` or `Seek` have returned ValNone. +func (p *pointIterator) Err() error { + return nil +} From ebd81c45db5a094fc1f454788cf206a4de4d1519 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Tue, 28 Nov 2023 04:52:20 +0300 Subject: [PATCH 070/112] feat(chstorage): query histograms too --- internal/chstorage/querier_metrics.go | 175 +++++++++++++++++++++++++- 1 file changed, 169 insertions(+), 6 deletions(-) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index e453f77c..741f4c75 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -3,6 +3,9 @@ package chstorage import ( "context" "fmt" + "maps" + "math" + "strconv" "strings" "time" @@ -10,9 +13,11 @@ import ( "github.com/ClickHouse/ch-go/proto" "github.com/go-faster/errors" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/value" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/annotations" + "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" "golang.org/x/sync/errgroup" @@ -183,6 +188,7 @@ type seriesKey struct { name string attributes string resource string + bucketKey [2]string } func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHints, matchers ...*labels.Matcher) (_ storage.SeriesSet, rerr error) { @@ -265,8 +271,9 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin } var ( - points []storage.Series - histSeries []storage.Series + points []storage.Series + histSeries []storage.Series + expHistSeries []storage.Series ) grp, grpCtx := errgroup.WithContext(ctx) grp.Go(func() error { @@ -287,23 +294,39 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin grp.Go(func() error { ctx := grpCtx - query, err := buildQuery(p.tables.ExpHistograms) + query, err := buildQuery(p.tables.Histograms) if err != nil { return err } - result, err := p.queryExpHistrograms(ctx, query) + result, err := p.queryHistograms(ctx, query) if err != nil { - return errors.Wrap(err, "query histrograms") + return errors.Wrap(err, "query histograms") } histSeries = result return nil }) + grp.Go(func() error { + ctx := grpCtx + + query, err := buildQuery(p.tables.ExpHistograms) + if err != nil { + return err + } + + result, err := p.queryExpHistograms(ctx, query) + if err != nil { + return errors.Wrap(err, "query exponential histograms") + } + expHistSeries = result + return nil + }) if err := grp.Wait(); err != nil { return nil, err } points = append(points, histSeries...) + points = append(points, expHistSeries...) return newSeriesSet(points), nil } @@ -376,7 +399,147 @@ func (p *promQuerier) queryPoints(ctx context.Context, query string) ([]storage. return result, nil } -func (p *promQuerier) queryExpHistrograms(ctx context.Context, query string) ([]storage.Series, error) { +func (p *promQuerier) queryHistograms(ctx context.Context, query string) ([]storage.Series, error) { + type seriesWithLabels struct { + series *series[pointData] + labels map[string]string + } + type histogramSample struct { + timestamp int64 + rawAttributes, rawResource string + attributes, resource map[string]string + flags pmetric.DataPointFlags + } + + var ( + set = map[seriesKey]seriesWithLabels{} + addSample = func( + name string, + val float64, + sample histogramSample, + bucketKey [2]string, + ) { + key := seriesKey{ + name: name, + attributes: sample.rawAttributes, + resource: sample.rawResource, + bucketKey: bucketKey, + } + s, ok := set[key] + if !ok { + s = seriesWithLabels{ + series: &series[pointData]{}, + labels: map[string]string{}, + } + set[key] = s + } + + if sample.flags.NoRecordedValue() { + val = math.Float64frombits(value.StaleNaN) + } + s.series.data.values = append(s.series.data.values, val) + s.series.ts = append(s.series.ts, sample.timestamp) + + s.labels["__name__"] = name + maps.Copy(s.labels, sample.attributes) + maps.Copy(s.labels, sample.resource) + if key := bucketKey[0]; key != "" { + s.labels[key] = bucketKey[1] + } + } + c = newHistogramColumns() + ) + if err := p.ch.Do(ctx, ch.Query{ + Body: query, + Result: c.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + for i := 0; i < c.timestamp.Rows(); i++ { + name := c.name.Row(i) + timestamp := c.timestamp.Row(i) + count := c.count.Row(i) + sum := c.sum.Row(i) + _min := c.min.Row(i) + _max := c.max.Row(i) + bucketCounts := c.bucketCounts.Row(i) + explicitBounds := c.explicitBounds.Row(i) + flags := pmetric.DataPointFlags(c.flags.Row(i)) + rawAttributes := c.attributes.Row(i) + rawResource := c.resource.Row(i) + + var ( + resource = map[string]string{} + attributes = map[string]string{} + ) + if err := parseLabels(rawResource, resource); err != nil { + return errors.Wrap(err, "parse resource") + } + if err := parseLabels(rawAttributes, attributes); err != nil { + return errors.Wrap(err, "parse attributes") + } + sample := histogramSample{ + timestamp: timestamp.UnixMilli(), + rawAttributes: rawAttributes, + rawResource: rawResource, + attributes: attributes, + resource: resource, + flags: flags, + } + + if sum.Set { + addSample(name+"_sum", sum.Value, sample, [2]string{}) + } + if _min.Set { + addSample(name+"_min", _min.Value, sample, [2]string{}) + } + if _max.Set { + addSample(name+"_max", _max.Value, sample, [2]string{}) + } + addSample(name+"_count", float64(count), sample, [2]string{}) + + var cumCount uint64 + for i := 0; i < min(len(bucketCounts), len(explicitBounds)); i++ { + bound := explicitBounds[i] + cumCount += bucketCounts[i] + + // Generate series with "_bucket" suffix and "le" label. + addSample("_bucket", float64(cumCount), sample, [2]string{ + "le", + strconv.FormatFloat(bound, 'f', -1, 64), + }) + } + + { + // Generate series with "_bucket" suffix and "le" label. + addSample("_bucket", float64(count), sample, [2]string{ + "le", + "+Inf", + }) + } + } + return nil + }, + }); err != nil { + return nil, errors.Wrap(err, "do query") + } + + var ( + result = make([]storage.Series, 0, len(set)) + lb labels.ScratchBuilder + ) + for _, s := range set { + lb.Reset() + for key, value := range s.labels { + lb.Add(key, value) + } + lb.Sort() + s.series.labels = lb.Labels() + result = append(result, s.series) + } + + return result, nil +} + +func (p *promQuerier) queryExpHistograms(ctx context.Context, query string) ([]storage.Series, error) { type seriesWithLabels struct { series *series[expHistData] labels map[string]string From cf069b0d01058e3499c85927215f8fb7f736a97f Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Tue, 28 Nov 2023 15:36:27 +0300 Subject: [PATCH 071/112] feat(chstorage.logs): undot --- integration/lokie2e/_testdata/logs.jsonl | 42 +++++------ integration/lokie2e/ch_test.go | 12 +++- integration/lokie2e/common_test.go | 90 ++++++++++++------------ internal/chstorage/columns.go | 6 ++ internal/chstorage/columns_logs.go | 39 +++++++++- internal/chstorage/inserter_logs.go | 42 ++++++++++- internal/chstorage/querier_logs.go | 74 ++++++++++++++++++- internal/chstorage/schema.go | 8 ++- internal/chstorage/schema_logs.go | 9 +++ internal/otelstorage/attrs.go | 7 ++ 10 files changed, 249 insertions(+), 80 deletions(-) diff --git a/integration/lokie2e/_testdata/logs.jsonl b/integration/lokie2e/_testdata/logs.jsonl index d64e5f87..eca2021f 100644 --- a/integration/lokie2e/_testdata/logs.jsonl +++ b/integration/lokie2e/_testdata/logs.jsonl @@ -1,21 +1,21 @@ -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770531506389037","observedTimeUnixNano":"1688770533485875260","body":{"stringValue":"{\"host\":\"182.232.235.224\", \"user-identifier\":\"murray3567\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"POST\", \"request\": \"/monetize\", \"protocol\":\"HTTP/2.0\", \"status\":503, \"bytes\":536, \"referer\": \"https://www.legacyeyeballs.io/streamline/mindshare/value-added/metrics\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532258117999","observedTimeUnixNano":"1688770533485879030","body":{"stringValue":"{\"host\":\"222.2.138.72\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"PUT\", \"request\": \"/strategize/e-services/24%2f7\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22430, \"referer\": \"https://www.globalapplications.info/scale/scalable/morph/collaborative\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532508421534","observedTimeUnixNano":"1688770533485879850","body":{"stringValue":"{\"host\":\"114.175.148.98\", \"user-identifier\":\"trantow3854\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PUT\", \"request\": \"/killer\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":9706, \"referer\": \"http://www.leadproactive.net/supply-chains\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506099582","observedTimeUnixNano":"1688770533485882410","body":{"stringValue":"{\"host\":\"224.56.79.52\", \"user-identifier\":\"armstrong8550\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"DELETE\", \"request\": \"/applications/incentivize/deliver/24%2f365\", \"protocol\":\"HTTP/1.0\", \"status\":501, \"bytes\":8709, \"referer\": \"http://www.directoptimize.biz/plug-and-play/leverage\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506357477","observedTimeUnixNano":"1688770533485884770","body":{"stringValue":"{\"host\":\"59.254.188.104\", \"user-identifier\":\"shanahan3762\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"PUT\", \"request\": \"/content/productize/expedite/viral\", \"protocol\":\"HTTP/1.0\", \"status\":204, \"bytes\":22262, \"referer\": \"https://www.internalone-to-one.io/productize\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506095582","observedTimeUnixNano":"1688770533485886960","body":{"stringValue":"{\"host\":\"108.166.198.19\", \"user-identifier\":\"rempel2418\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"PUT\", \"request\": \"/infrastructures/dynamic/interfaces\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":12418, \"referer\": \"http://www.directutilize.info/revolutionize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506270749","observedTimeUnixNano":"1688770533485888960","body":{"stringValue":"{\"host\":\"32.172.191.106\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"PATCH\", \"request\": \"/next-generation/efficient\", \"protocol\":\"HTTP/1.0\", \"status\":301, \"bytes\":19187, \"referer\": \"http://www.humanarchitectures.org/content/cross-media/brand/extensible\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"301"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506356337","observedTimeUnixNano":"1688770533485891020","body":{"stringValue":"{\"host\":\"245.62.143.62\", \"user-identifier\":\"kshlerin1534\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"GET\", \"request\": \"/architect/portals/niches\", \"protocol\":\"HTTP/1.0\", \"status\":403, \"bytes\":19870, \"referer\": \"http://www.directtransition.biz/mesh/next-generation/roi\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532007261105","observedTimeUnixNano":"1688770533485893360","body":{"stringValue":"{\"host\":\"112.190.183.173\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"GET\", \"request\": \"/dynamic/sticky\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":21741, \"referer\": \"https://www.humaninfrastructures.biz/users/collaborative/proactive/proactive\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"201"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506066453","observedTimeUnixNano":"1688770533485897459","body":{"stringValue":"{\"host\":\"117.12.231.81\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"PUT\", \"request\": \"/models/empower/open-source\", \"protocol\":\"HTTP/1.1\", \"status\":302, \"bytes\":27722, \"referer\": \"https://www.dynamicmindshare.org/e-business/intuitive/unleash/markets\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506070283","observedTimeUnixNano":"1688770533485899839","body":{"stringValue":"{\"host\":\"236.7.233.166\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"HEAD\", \"request\": \"/enterprise\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":20357, \"referer\": \"https://www.investorseize.net/cultivate/synergies/revolutionize/extend\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506277879","observedTimeUnixNano":"1688770533485900339","body":{"stringValue":"{\"host\":\"55.119.244.237\", \"user-identifier\":\"buckridge5168\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"HEAD\", \"request\": \"/sexy/synthesize/architectures/front-end\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":24908, \"referer\": \"http://www.nationale-markets.net/bricks-and-clicks\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531756857070","observedTimeUnixNano":"1688770533485902439","body":{"stringValue":"{\"host\":\"14.182.107.209\", \"user-identifier\":\"kozey6276\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"DELETE\", \"request\": \"/embrace/cutting-edge/empower/empower\", \"protocol\":\"HTTP/1.0\", \"status\":405, \"bytes\":19380, \"referer\": \"http://www.legacycompelling.name/drive/strategic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506329078","observedTimeUnixNano":"1688770533485904529","body":{"stringValue":"{\"host\":\"80.210.177.228\", \"user-identifier\":\"nitzsche3256\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"GET\", \"request\": \"/paradigms/innovate/b2b/drive\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":923, \"referer\": \"http://www.legacyend-to-end.io/enhance\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506331908","observedTimeUnixNano":"1688770533485906569","body":{"stringValue":"{\"host\":\"86.173.91.15\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"PATCH\", \"request\": \"/open-source/integrate/open-source/visualize\", \"protocol\":\"HTTP/2.0\", \"status\":200, \"bytes\":19653, \"referer\": \"https://www.principalgenerate.io/visualize/dot-com\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506091172","observedTimeUnixNano":"1688770533485908829","body":{"stringValue":"{\"host\":\"214.52.159.245\", \"user-identifier\":\"lowe8277\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"POST\", \"request\": \"/transition/synthesize/applications/24%2f365\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":13965, \"referer\": \"https://www.leadnext-generation.info/redefine/e-business\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506274739","observedTimeUnixNano":"1688770533485910989","body":{"stringValue":"{\"host\":\"79.228.108.32\", \"user-identifier\":\"sanford2465\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"POST\", \"request\": \"/incentivize\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":9074, \"referer\": \"http://www.chiefe-business.info/e-enable/incentivize/transition\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506330508","observedTimeUnixNano":"1688770533485913019","body":{"stringValue":"{\"host\":\"153.126.163.17\", \"user-identifier\":\"terry2422\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"DELETE\", \"request\": \"/e-enable\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":1033, \"referer\": \"http://www.customere-commerce.info/mesh\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506406776","observedTimeUnixNano":"1688770533485915089","body":{"stringValue":"{\"host\":\"106.153.80.249\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"POST\", \"request\": \"/systems\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":26505, \"referer\": \"http://www.chiefsolutions.info/technologies/b2c/embrace/applications\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506063113","observedTimeUnixNano":"1688770533485917269","body":{"stringValue":"{\"host\":\"139.211.170.136\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:26 +0000\", \"method\": \"GET\", \"request\": \"/iterate/implement/relationships\", \"protocol\":\"HTTP/1.1\", \"status\":100, \"bytes\":22426, \"referer\": \"https://www.legacyseamless.io/transparent\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506072323","observedTimeUnixNano":"1688770533485920149","body":{"stringValue":"{\"host\":\"159.238.229.29\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"POST\", \"request\": \"/world-class\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":22475, \"referer\": \"https://www.nationalend-to-end.io/cross-platform/visualize/compelling/virtual\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506267519","observedTimeUnixNano":"1688770533485920629","body":{"stringValue":"{\"host\":\"240.39.214.21\", \"user-identifier\":\"yost2184\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"POST\", \"request\": \"/initiatives/benchmark/mesh/utilize\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":6318, \"referer\": \"http://www.regionalreintermediate.biz/innovative/utilize/morph\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770532759320597","observedTimeUnixNano":"1688770534505475760","body":{"stringValue":"{\"host\":\"219.253.131.24\", \"user-identifier\":\"nolan2452\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"POST\", \"request\": \"/transform/partnerships/compelling\", \"protocol\":\"HTTP/1.0\", \"status\":504, \"bytes\":5432, \"referer\": \"http://www.dynamicpartnerships.info/distributed/e-enable\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"504"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533009722650","observedTimeUnixNano":"1688770534505481330","body":{"stringValue":"{\"host\":\"140.150.10.211\", \"user-identifier\":\"stark5567\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PUT\", \"request\": \"/visionary/harness/exploit\", \"protocol\":\"HTTP/1.0\", \"status\":416, \"bytes\":12222, \"referer\": \"https://www.internalvortals.io/initiatives/niches\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533260610181","observedTimeUnixNano":"1688770534505485390","body":{"stringValue":"{\"host\":\"255.39.188.234\", \"user-identifier\":\"cartwright7210\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PATCH\", \"request\": \"/best-of-breed/transparent\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":28207, \"referer\": \"https://www.productvertical.com/models/harness/bandwidth/scalable\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533511255568","observedTimeUnixNano":"1688770534505488520","body":{"stringValue":"{\"host\":\"32.3.226.175\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"PUT\", \"request\": \"/communities/evolve/content/interfaces\", \"protocol\":\"HTTP/1.0\", \"status\":200, \"bytes\":16759, \"referer\": \"https://www.globalnext-generation.info/seize/engage/unleash/empower\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533761830695","observedTimeUnixNano":"1688770534505491520","body":{"stringValue":"{\"host\":\"224.24.229.231\", \"user-identifier\":\"runte3687\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"POST\", \"request\": \"/e-business/brand/methodologies\", \"protocol\":\"HTTP/2.0\", \"status\":416, \"bytes\":13103, \"referer\": \"https://www.regionalrich.com/action-items\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534012191357","observedTimeUnixNano":"1688770534505494330","body":{"stringValue":"{\"host\":\"34.60.133.32\", \"user-identifier\":\"wisoky3066\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"PUT\", \"request\": \"/out-of-the-box\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":5665, \"referer\": \"http://www.leadrevolutionize.org/efficient/partnerships/architect/solutions\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534263072946","observedTimeUnixNano":"1688770534505497230","body":{"stringValue":"{\"host\":\"138.73.65.106\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"HEAD\", \"request\": \"/deploy/front-end/vertical/metrics\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":28841, \"referer\": \"https://www.regionalsyndicate.name/solutions\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770535014595986","observedTimeUnixNano":"1688770535605373502","body":{"stringValue":"{\"host\":\"119.217.207.214\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"POST\", \"request\": \"/magnetic/reinvent/virtual\", \"protocol\":\"HTTP/2.0\", \"status\":204, \"bytes\":23155, \"referer\": \"http://www.leadbandwidth.com/brand/unleash\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770535265453194","observedTimeUnixNano":"1688770535605378152","body":{"stringValue":"{\"host\":\"20.82.163.90\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"GET\", \"request\": \"/seamless/seamless/recontextualize\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":7620, \"referer\": \"http://www.leaduser-centric.io/reinvent/engineer/experiences\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770535515889000","observedTimeUnixNano":"1688770535605381162","body":{"stringValue":"{\"host\":\"75.110.63.232\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PUT\", \"request\": \"/one-to-one/aggregate\", \"protocol\":\"HTTP/1.1\", \"status\":416, \"bytes\":21091, \"referer\": \"https://www.regionalimplement.name/relationships/engage/e-services/redefine\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534514124542","observedTimeUnixNano":"1688770535605383892","body":{"stringValue":"{\"host\":\"150.204.109.167\", \"user-identifier\":\"swift5175\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"HEAD\", \"request\": \"/integrated/distributed/methodologies\", \"protocol\":\"HTTP/2.0\", \"status\":404, \"bytes\":10857, \"referer\": \"https://www.seniorsexy.com/distributed\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534764273486","observedTimeUnixNano":"1688770535605386732","body":{"stringValue":"{\"host\":\"15.176.190.242\", \"user-identifier\":\"champlin5148\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"PUT\", \"request\": \"/monetize/mindshare/rich/extensible\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":17396, \"referer\": \"http://www.globalrepurpose.com/redefine/brand/architect\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770535766768587","observedTimeUnixNano":"1688770536805705482","body":{"stringValue":"{\"host\":\"159.240.99.65\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"GET\", \"request\": \"/seamless\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":4573, \"referer\": \"https://www.forwardgenerate.org/revolutionize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536017199113","observedTimeUnixNano":"1688770536805710712","body":{"stringValue":"{\"host\":\"254.246.88.15\", \"user-identifier\":\"feest4001\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PUT\", \"request\": \"/architectures\", \"protocol\":\"HTTP/1.1\", \"status\":504, \"bytes\":9703, \"referer\": \"http://www.seniorgenerate.name/roi/web-enabled/mesh\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"504"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536267969591","observedTimeUnixNano":"1688770536805714342","body":{"stringValue":"{\"host\":\"117.113.156.145\", \"user-identifier\":\"hansen6831\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PATCH\", \"request\": \"/best-of-breed/strategize/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":500, \"bytes\":7590, \"referer\": \"http://www.internalnext-generation.biz/granular/relationships\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536518702679","observedTimeUnixNano":"1688770536805717532","body":{"stringValue":"{\"host\":\"108.138.59.166\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"GET\", \"request\": \"/proactive\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":20470, \"referer\": \"https://www.corporatevisionary.org/models/integrate/supply-chains/visualize\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"502"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536769226151","observedTimeUnixNano":"1688770536805720572","body":{"stringValue":"{\"host\":\"80.21.65.147\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"POST\", \"request\": \"/methodologies\", \"protocol\":\"HTTP/1.1\", \"status\":504, \"bytes\":24809, \"referer\": \"https://www.leade-tailers.io/front-end/turn-key/supply-chains\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"504"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770537019590646","observedTimeUnixNano":"1688770538105581695","body":{"stringValue":"{\"host\":\"48.48.132.74\", \"user-identifier\":\"sauer4564\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"PATCH\", \"request\": \"/ubiquitous/transparent\", \"protocol\":\"HTTP/1.1\", \"status\":204, \"bytes\":21351, \"referer\": \"http://www.corporatebrand.com/synergies\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"204"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537270539598","observedTimeUnixNano":"1688770538105586155","body":{"stringValue":"{\"host\":\"41.84.80.177\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"POST\", \"request\": \"/schemas\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":9416, \"referer\": \"http://www.forwardrich.name/bleeding-edge\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537520765634","observedTimeUnixNano":"1688770538105589205","body":{"stringValue":"{\"host\":\"215.39.102.24\", \"user-identifier\":\"pouros8556\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"HEAD\", \"request\": \"/infrastructures\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":3738, \"referer\": \"https://www.internationalrevolutionary.io/incentivize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537771653837","observedTimeUnixNano":"1688770538105592045","body":{"stringValue":"{\"host\":\"233.20.118.85\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"PATCH\", \"request\": \"/bandwidth/whiteboard/relationships/b2b\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":13768, \"referer\": \"https://www.internalchannels.io/synthesize/customized/solutions/target\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538022089288","observedTimeUnixNano":"1688770538105594684","body":{"stringValue":"{\"host\":\"76.247.19.239\", \"user-identifier\":\"cormier5083\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"DELETE\", \"request\": \"/interfaces\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":4639, \"referer\": \"https://www.principalengineer.biz/solutions/synergistic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770538272986089","observedTimeUnixNano":"1688770539304866280","body":{"stringValue":"{\"host\":\"182.27.181.198\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"HEAD\", \"request\": \"/deliver/e-tailers\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":13889, \"referer\": \"http://www.investorextensible.com/empower/clicks-and-mortar/dot-com\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538523374810","observedTimeUnixNano":"1688770539304871430","body":{"stringValue":"{\"host\":\"135.212.14.226\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/integrate/e-business/orchestrate\", \"protocol\":\"HTTP/1.1\", \"status\":416, \"bytes\":25334, \"referer\": \"http://www.investordynamic.name/benchmark/mission-critical\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538774176503","observedTimeUnixNano":"1688770539304874460","body":{"stringValue":"{\"host\":\"47.20.233.211\", \"user-identifier\":\"schowalter7876\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/bricks-and-clicks/channels\", \"protocol\":\"HTTP/2.0\", \"status\":204, \"bytes\":11038, \"referer\": \"http://www.dynamicmission-critical.info/eyeballs\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539275422243","observedTimeUnixNano":"1688770539304875030","body":{"stringValue":"{\"host\":\"251.240.185.7\", \"user-identifier\":\"kirlin6527\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/embrace/next-generation/value-added\", \"protocol\":\"HTTP/1.0\", \"status\":204, \"bytes\":2768, \"referer\": \"http://www.leadembrace.io/brand\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539024541514","observedTimeUnixNano":"1688770539304877770","body":{"stringValue":"{\"host\":\"225.112.239.87\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"PATCH\", \"request\": \"/vertical/customized\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":25553, \"referer\": \"https://www.districtenterprise.net/schemas/drive\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"500"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770539525776242","observedTimeUnixNano":"1688770540604713526","body":{"stringValue":"{\"host\":\"110.213.172.140\", \"user-identifier\":\"flatley2072\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"POST\", \"request\": \"/back-end/plug-and-play\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":1079, \"referer\": \"https://www.corporatesyndicate.net/proactive/end-to-end/out-of-the-box/partnerships\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539776638361","observedTimeUnixNano":"1688770540604718996","body":{"stringValue":"{\"host\":\"132.59.120.128\", \"user-identifier\":\"kautzer2610\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"GET\", \"request\": \"/embrace/mission-critical\", \"protocol\":\"HTTP/2.0\", \"status\":403, \"bytes\":9174, \"referer\": \"https://www.humanengineer.name/engineer/value-added/interactive\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540027065299","observedTimeUnixNano":"1688770540604723106","body":{"stringValue":"{\"host\":\"255.128.41.196\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"GET\", \"request\": \"/dynamic\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":23462, \"referer\": \"http://www.seniorenterprise.net/best-of-breed/grow/cross-media\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540277926986","observedTimeUnixNano":"1688770540604726566","body":{"stringValue":"{\"host\":\"108.113.41.211\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"HEAD\", \"request\": \"/cutting-edge/deliverables/cross-media\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":18274, \"referer\": \"https://www.directvisualize.io/vortals/24/7\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540528287773","observedTimeUnixNano":"1688770540604729666","body":{"stringValue":"{\"host\":\"126.219.241.28\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"GET\", \"request\": \"/expedite/engineer\", \"protocol\":\"HTTP/2.0\", \"status\":503, \"bytes\":9425, \"referer\": \"http://www.internalrecontextualize.info/content/facilitate\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770540779141751","observedTimeUnixNano":"1688770541805060126","body":{"stringValue":"{\"host\":\"75.100.204.75\", \"user-identifier\":\"stiedemann6660\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"PATCH\", \"request\": \"/incubate/syndicate/communities/enhance\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":28778, \"referer\": \"https://www.principalmarkets.biz/web-readiness/holistic\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541029484568","observedTimeUnixNano":"1688770541805067016","body":{"stringValue":"{\"host\":\"164.61.170.176\", \"user-identifier\":\"feil3053\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"PUT\", \"request\": \"/deliver\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":12408, \"referer\": \"https://www.seniordot-com.net/best-of-breed/wireless/partnerships/magnetic\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"500"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541280380253","observedTimeUnixNano":"1688770541805070706","body":{"stringValue":"{\"host\":\"237.229.232.170\", \"user-identifier\":\"cummings3317\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"HEAD\", \"request\": \"/technologies\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":23099, \"referer\": \"https://www.nationalsynergistic.net/cross-media/frictionless\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541531131440","observedTimeUnixNano":"1688770541805073826","body":{"stringValue":"{\"host\":\"191.42.120.4\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"DELETE\", \"request\": \"/cultivate/enterprise\", \"protocol\":\"HTTP/1.1\", \"status\":503, \"bytes\":1096, \"referer\": \"http://www.regionalend-to-end.com/e-commerce/bricks-and-clicks\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541781600583","observedTimeUnixNano":"1688770541805077586","body":{"stringValue":"{\"host\":\"115.72.122.202\", \"user-identifier\":\"swift7514\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"PATCH\", \"request\": \"/transition\", \"protocol\":\"HTTP/2.0\", \"status\":405, \"bytes\":2870, \"referer\": \"https://www.leadenvisioneer.biz/compelling/killer\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770542031960818","observedTimeUnixNano":"1688770543105018384","body":{"stringValue":"{\"host\":\"125.249.12.227\", \"user-identifier\":\"crona1635\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"GET\", \"request\": \"/schemas\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":23913, \"referer\": \"https://www.leadinteractive.name/synthesize/deliverables/unleash/viral\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542784196884","observedTimeUnixNano":"1688770543105020304","body":{"stringValue":"{\"host\":\"31.184.47.132\", \"user-identifier\":\"boyer7325\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"GET\", \"request\": \"/24%2f7/web+services/extensible/reintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":17233, \"referer\": \"https://www.leadredefine.com/morph/exploit/seize/whiteboard\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542282846871","observedTimeUnixNano":"1688770543105023974","body":{"stringValue":"{\"host\":\"180.227.124.253\", \"user-identifier\":\"glover2525\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"HEAD\", \"request\": \"/roi/revolutionary\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":4997, \"referer\": \"https://www.investorrecontextualize.net/cross-platform/syndicate/initiatives/content\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542533294682","observedTimeUnixNano":"1688770543105026754","body":{"stringValue":"{\"host\":\"35.167.110.144\", \"user-identifier\":\"kiehn6344\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"PATCH\", \"request\": \"/user-centric/incubate/disintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":503, \"bytes\":19773, \"referer\": \"https://www.investorvertical.info/drive/streamline/e-services/reinvent\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543034516918","observedTimeUnixNano":"1688770543105029564","body":{"stringValue":"{\"host\":\"135.76.24.123\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"GET\", \"request\": \"/infrastructures\", \"protocol\":\"HTTP/1.1\", \"status\":501, \"bytes\":29365, \"referer\": \"http://www.dynamicweb-enabled.com/web-readiness/markets/revolutionize\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770543285385529","observedTimeUnixNano":"1688770544305313540","body":{"stringValue":"{\"host\":\"204.141.204.194\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"DELETE\", \"request\": \"/rich\", \"protocol\":\"HTTP/1.1\", \"status\":401, \"bytes\":6762, \"referer\": \"https://www.principaldeliver.biz/reintermediate/channels/impactful\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543535766279","observedTimeUnixNano":"1688770544305318969","body":{"stringValue":"{\"host\":\"25.50.100.83\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"DELETE\", \"request\": \"/sexy/web-readiness/infomediaries\", \"protocol\":\"HTTP/2.0\", \"status\":400, \"bytes\":8067, \"referer\": \"https://www.customermethodologies.com/user-centric/schemas/deliverables/synergize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543786551452","observedTimeUnixNano":"1688770544305322659","body":{"stringValue":"{\"host\":\"172.52.222.78\", \"user-identifier\":\"waelchi8475\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"PUT\", \"request\": \"/leverage/syndicate\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":13351, \"referer\": \"http://www.dynamicweb-readiness.net/vertical/holistic/magnetic/content\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544037029250","observedTimeUnixNano":"1688770544305326179","body":{"stringValue":"{\"host\":\"18.33.245.88\", \"user-identifier\":\"brekke5341\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"HEAD\", \"request\": \"/world-class\", \"protocol\":\"HTTP/1.1\", \"status\":100, \"bytes\":23594, \"referer\": \"https://www.districtrevolutionary.org/brand\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544287848500","observedTimeUnixNano":"1688770544305331319","body":{"stringValue":"{\"host\":\"95.146.146.56\", \"user-identifier\":\"witting8863\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"POST\", \"request\": \"/content\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22835, \"referer\": \"https://www.internationalb2c.org/recontextualize/recontextualize/grow/transform\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"502"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770545540651551","observedTimeUnixNano":"1688770545605162902","body":{"stringValue":"{\"host\":\"72.242.71.239\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"POST\", \"request\": \"/global\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":3026, \"referer\": \"http://www.nationalinfomediaries.net/solutions/roi/innovative\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544538215259","observedTimeUnixNano":"1688770545605168132","body":{"stringValue":"{\"host\":\"221.31.17.132\", \"user-identifier\":\"hammes8155\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"PUT\", \"request\": \"/innovate\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":18182, \"referer\": \"http://www.districtempower.name/architectures\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544789083788","observedTimeUnixNano":"1688770545605171772","body":{"stringValue":"{\"host\":\"73.25.185.22\", \"user-identifier\":\"schaefer3552\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"HEAD\", \"request\": \"/collaborative/vertical\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":13514, \"referer\": \"https://www.legacyproductize.net/interfaces\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545039444047","observedTimeUnixNano":"1688770545605175002","body":{"stringValue":"{\"host\":\"133.159.101.254\", \"user-identifier\":\"bernier7784\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"POST\", \"request\": \"/deploy/synergize\", \"protocol\":\"HTTP/1.1\", \"status\":405, \"bytes\":4079, \"referer\": \"http://www.internalinterfaces.io/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545290299404","observedTimeUnixNano":"1688770545605178022","body":{"stringValue":"{\"host\":\"63.166.3.113\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"PUT\", \"request\": \"/networks/e-enable\", \"protocol\":\"HTTP/2.0\", \"status\":301, \"bytes\":23413, \"referer\": \"http://www.leadcontent.info/portals/content/transparent\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"301"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770546292750429","observedTimeUnixNano":"1688770546805390176","body":{"stringValue":"{\"host\":\"49.154.204.143\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"PATCH\", \"request\": \"/solutions\", \"protocol\":\"HTTP/2.0\", \"status\":405, \"bytes\":12772, \"referer\": \"https://www.chieftechnologies.com/deliver\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546543451957","observedTimeUnixNano":"1688770546805394616","body":{"stringValue":"{\"host\":\"139.113.227.72\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"PATCH\", \"request\": \"/users/dynamic/revolutionary\", \"protocol\":\"HTTP/1.1\", \"status\":203, \"bytes\":11927, \"referer\": \"http://www.investorsynergies.org/leading-edge/productize/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546793929399","observedTimeUnixNano":"1688770546805397786","body":{"stringValue":"{\"host\":\"18.225.144.149\", \"user-identifier\":\"macejkovic4811\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"DELETE\", \"request\": \"/cross-platform/optimize/e-tailers\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":8359, \"referer\": \"https://www.districtmesh.name/channels/partnerships/utilize/plug-and-play\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545791539778","observedTimeUnixNano":"1688770546805401576","body":{"stringValue":"{\"host\":\"113.112.205.16\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"DELETE\", \"request\": \"/generate/schemas/impactful/frictionless\", \"protocol\":\"HTTP/1.0\", \"status\":501, \"bytes\":2572, \"referer\": \"https://www.chiefiterate.info/unleash\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"501"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546041916043","observedTimeUnixNano":"1688770546805404055","body":{"stringValue":"{\"host\":\"147.251.113.164\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"GET\", \"request\": \"/integrated\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":10412, \"referer\": \"http://www.productrobust.io/strategic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770547044430211","observedTimeUnixNano":"1688770548105337580","body":{"stringValue":"{\"host\":\"31.12.63.70\", \"user-identifier\":\"carroll3128\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"POST\", \"request\": \"/value-added/infomediaries\", \"protocol\":\"HTTP/1.1\", \"status\":201, \"bytes\":15247, \"referer\": \"https://www.principalopen-source.name/turn-key/facilitate/orchestrate\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547295243595","observedTimeUnixNano":"1688770548105342880","body":{"stringValue":"{\"host\":\"160.136.74.178\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"PUT\", \"request\": \"/productize/reintermediate/markets\", \"protocol\":\"HTTP/1.0\", \"status\":100, \"bytes\":28739, \"referer\": \"http://www.seniorplatforms.name/infrastructures\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"100"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547545668186","observedTimeUnixNano":"1688770548105346740","body":{"stringValue":"{\"host\":\"218.137.103.90\", \"user-identifier\":\"huels6041\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"POST\", \"request\": \"/enhance\", \"protocol\":\"HTTP/1.0\", \"status\":401, \"bytes\":1115, \"referer\": \"https://www.internalrevolutionary.com/roi/productize\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547796500899","observedTimeUnixNano":"1688770548105350040","body":{"stringValue":"{\"host\":\"152.7.8.228\", \"user-identifier\":\"mosciski3102\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"PUT\", \"request\": \"/matrix/content\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":9350, \"referer\": \"http://www.humanreintermediate.com/architectures/web-enabled/24/365\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548046805313","observedTimeUnixNano":"1688770548105353230","body":{"stringValue":"{\"host\":\"189.145.183.88\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"HEAD\", \"request\": \"/mesh/b2b/syndicate/technologies\", \"protocol\":\"HTTP/1.1\", \"status\":200, \"bytes\":23769, \"referer\": \"http://www.humanparadigms.info/world-class\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770548548089763","observedTimeUnixNano":"1688770549305574358","body":{"stringValue":"{\"host\":\"149.219.8.124\", \"user-identifier\":\"johnston5032\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"DELETE\", \"request\": \"/convergence\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":20805, \"referer\": \"http://www.dynamicuser-centric.com/schemas/sexy/granular/bricks-and-clicks\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548798941415","observedTimeUnixNano":"1688770549305580998","body":{"stringValue":"{\"host\":\"202.159.240.173\", \"user-identifier\":\"hayes4575\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"POST\", \"request\": \"/cross-platform/content/functionalities/repurpose\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":21455, \"referer\": \"http://www.regionalgenerate.info/b2c/vortals/solutions\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549049313785","observedTimeUnixNano":"1688770549305588108","body":{"stringValue":"{\"host\":\"120.119.68.91\", \"user-identifier\":\"heidenreich7556\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"HEAD\", \"request\": \"/aggregate/networks\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":26555, \"referer\": \"http://www.dynamicvisionary.name/scalable\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549300083475","observedTimeUnixNano":"1688770549305592998","body":{"stringValue":"{\"host\":\"23.194.158.90\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"PATCH\", \"request\": \"/revolutionize/envisioneer/supply-chains\", \"protocol\":\"HTTP/1.1\", \"status\":302, \"bytes\":24515, \"referer\": \"https://www.dynamicproactive.io/customized/deliverables\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548297678124","observedTimeUnixNano":"1688770549305597048","body":{"stringValue":"{\"host\":\"171.235.211.64\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"PUT\", \"request\": \"/proactive/expedite/seize/leading-edge\", \"protocol\":\"HTTP/1.1\", \"status\":403, \"bytes\":16631, \"referer\": \"https://www.humanengineer.info/extend\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770549550496684","observedTimeUnixNano":"1688770550605255542","body":{"stringValue":"{\"host\":\"35.243.14.54\", \"user-identifier\":\"weimann5622\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"DELETE\", \"request\": \"/seamless\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":27406, \"referer\": \"https://www.regionalembrace.biz/e-business/24/7\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550552786577","observedTimeUnixNano":"1688770550605257182","body":{"stringValue":"{\"host\":\"206.39.201.209\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"DELETE\", \"request\": \"/transform/infrastructures\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":17105, \"referer\": \"https://www.leadsticky.info/wireless/one-to-one\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549801322883","observedTimeUnixNano":"1688770550605263581","body":{"stringValue":"{\"host\":\"101.240.9.159\", \"user-identifier\":\"turcotte5453\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"GET\", \"request\": \"/revolutionary/revolutionize/incubate/reintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":10229, \"referer\": \"http://www.forwardnetworks.org/generate/exploit/b2c/web services\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"406"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550051625492","observedTimeUnixNano":"1688770550605268311","body":{"stringValue":"{\"host\":\"148.16.104.181\", \"user-identifier\":\"bednar5574\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"GET\", \"request\": \"/collaborative/seize\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":13672, \"referer\": \"https://www.internationalinfrastructures.io/optimize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550302524748","observedTimeUnixNano":"1688770550605271551","body":{"stringValue":"{\"host\":\"12.50.238.58\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"DELETE\", \"request\": \"/value-added/supply-chains\", \"protocol\":\"HTTP/1.0\", \"status\":200, \"bytes\":19948, \"referer\": \"http://www.investorefficient.biz/intuitive/24/7\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"200"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770550803736612","observedTimeUnixNano":"1688770551805598004","body":{"stringValue":"{\"host\":\"6.57.16.190\", \"user-identifier\":\"koelpin1807\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"HEAD\", \"request\": \"/mesh\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":10734, \"referer\": \"http://www.regionalbandwidth.com/integrate/benchmark\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551054058149","observedTimeUnixNano":"1688770551805607224","body":{"stringValue":"{\"host\":\"77.236.31.140\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"PATCH\", \"request\": \"/mission-critical\", \"protocol\":\"HTTP/2.0\", \"status\":201, \"bytes\":22561, \"referer\": \"https://www.directmaximize.com/supply-chains/deliverables/web-enabled\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551305051212","observedTimeUnixNano":"1688770551805613764","body":{"stringValue":"{\"host\":\"96.170.101.152\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"GET\", \"request\": \"/implement/cultivate/revolutionary\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":19505, \"referer\": \"http://www.directrobust.org/dot-com/benchmark\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551555515704","observedTimeUnixNano":"1688770551805620524","body":{"stringValue":"{\"host\":\"250.164.115.76\", \"user-identifier\":\"hermiston4072\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PUT\", \"request\": \"/experiences\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":18299, \"referer\": \"http://www.directrobust.name/initiatives\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770551806068175","observedTimeUnixNano":"1688770552904845153","body":{"stringValue":"{\"host\":\"254.168.97.108\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PUT\", \"request\": \"/vertical/robust/incubate\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":12490, \"referer\": \"http://www.productweb services.name/redefine/engage/bleeding-edge/maximize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PUT"}},{"key":"http_status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552056479008","observedTimeUnixNano":"1688770552904851803","body":{"stringValue":"{\"host\":\"118.204.193.76\", \"user-identifier\":\"kihn1514\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"DELETE\", \"request\": \"/dot-com/scalable/interfaces\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":1951, \"referer\": \"https://www.dynamicleading-edge.org/customized/deliverables\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"502"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552808876801","observedTimeUnixNano":"1688770552904853303","body":{"stringValue":"{\"host\":\"239.84.230.236\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"DELETE\", \"request\": \"/back-end/target\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":13592, \"referer\": \"http://www.corporatefacilitate.io/orchestrate/global\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552306744053","observedTimeUnixNano":"1688770552904857773","body":{"stringValue":"{\"host\":\"11.118.181.255\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PATCH\", \"request\": \"/one-to-one/channels/networks/collaborative\", \"protocol\":\"HTTP/1.1\", \"status\":501, \"bytes\":25565, \"referer\": \"http://www.dynamicinterfaces.biz/deliverables/real-time/global\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552557804932","observedTimeUnixNano":"1688770552904866042","body":{"stringValue":"{\"host\":\"38.157.64.60\", \"user-identifier\":\"collins3161\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"POST\", \"request\": \"/leverage/convergence\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":7451, \"referer\": \"http://www.dynamicapplications.name/scalable\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770553059904090","observedTimeUnixNano":"1688770554104914869","body":{"stringValue":"{\"host\":\"91.210.58.64\", \"user-identifier\":\"hirthe5885\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"POST\", \"request\": \"/experiences\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":26372, \"referer\": \"https://www.globalreintermediate.biz/scalable/strategic/deliver\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553310125793","observedTimeUnixNano":"1688770554104921949","body":{"stringValue":"{\"host\":\"171.146.184.207\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"HEAD\", \"request\": \"/global/wireless/initiatives\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":3559, \"referer\": \"http://www.districtextensible.info/sexy\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553561204179","observedTimeUnixNano":"1688770554104926178","body":{"stringValue":"{\"host\":\"223.135.210.193\", \"user-identifier\":\"lakin5175\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"DELETE\", \"request\": \"/e-commerce\", \"protocol\":\"HTTP/1.1\", \"status\":301, \"bytes\":17589, \"referer\": \"http://www.investortransform.org/embrace\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"301"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553811251136","observedTimeUnixNano":"1688770554104929478","body":{"stringValue":"{\"host\":\"167.99.122.25\", \"user-identifier\":\"dietrich4410\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"DELETE\", \"request\": \"/visualize/impactful/bleeding-edge/portals\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":21701, \"referer\": \"http://www.seniorredefine.info/e-commerce\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554061484609","observedTimeUnixNano":"1688770554104932588","body":{"stringValue":"{\"host\":\"12.8.67.46\", \"user-identifier\":\"stracke8218\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"HEAD\", \"request\": \"/ubiquitous/morph/visualize\", \"protocol\":\"HTTP/1.0\", \"status\":401, \"bytes\":3223, \"referer\": \"http://www.regionaldisintermediate.name/communities/synergies/drive\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770555062845794","observedTimeUnixNano":"1688770555404999753","body":{"stringValue":"{\"host\":\"68.65.52.180\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"PATCH\", \"request\": \"/engage/web-enabled/virtual/metrics\", \"protocol\":\"HTTP/2.0\", \"status\":100, \"bytes\":22935, \"referer\": \"http://www.dynamicfront-end.info/revolutionize/architect\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554311549923","observedTimeUnixNano":"1688770555405006503","body":{"stringValue":"{\"host\":\"58.176.246.52\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"HEAD\", \"request\": \"/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":26702, \"referer\": \"http://www.corporateniches.biz/exploit\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"400"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770555313060774","observedTimeUnixNano":"1688770555405007793","body":{"stringValue":"{\"host\":\"219.158.106.54\", \"user-identifier\":\"treutel4445\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"HEAD\", \"request\": \"/granular/killer/enhance/leverage\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":7060, \"referer\": \"http://www.productmonetize.org/visualize/paradigms/orchestrate/seize\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"HEAD"}},{"key":"http_status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554561762855","observedTimeUnixNano":"1688770555405012603","body":{"stringValue":"{\"host\":\"89.109.67.122\", \"user-identifier\":\"lang1745\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"PATCH\", \"request\": \"/syndicate\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":4309, \"referer\": \"http://www.chiefintuitive.biz/sexy/generate/end-to-end\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554812778300","observedTimeUnixNano":"1688770555405016873","body":{"stringValue":"{\"host\":\"108.149.26.184\", \"user-identifier\":\"wyman8181\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"DELETE\", \"request\": \"/infrastructures/utilize/mindshare\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":8395, \"referer\": \"https://www.dynamicintuitive.biz/efficient/eyeballs/seamless/enable\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770555564076188","observedTimeUnixNano":"1688770556605475977","body":{"stringValue":"{\"host\":\"214.112.255.154\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"POST\", \"request\": \"/metrics/24%2f7/deliverables/technologies\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":8201, \"referer\": \"http://www.centralfacilitate.net/convergence/repurpose\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"POST"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770555814316806","observedTimeUnixNano":"1688770556605481157","body":{"stringValue":"{\"host\":\"222.210.64.103\", \"user-identifier\":\"jaskolski1751\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"HEAD\", \"request\": \"/synergistic\", \"protocol\":\"HTTP/2.0\", \"status\":500, \"bytes\":1213, \"referer\": \"http://www.internationalbenchmark.info/cross-platform/infomediaries\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556065384879","observedTimeUnixNano":"1688770556605484527","body":{"stringValue":"{\"host\":\"86.90.114.24\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"GET\", \"request\": \"/synergize/incubate/platforms\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":8681, \"referer\": \"http://www.forwardvisualize.info/technologies/holistic\"}"},"attributes":[{"key":"http_status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556315409620","observedTimeUnixNano":"1688770556605487627","body":{"stringValue":"{\"host\":\"234.206.18.83\", \"user-identifier\":\"kozey6774\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"GET\", \"request\": \"/synthesize/users/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":500, \"bytes\":24463, \"referer\": \"http://www.legacyplug-and-play.net/mesh\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556566005240","observedTimeUnixNano":"1688770556605490717","body":{"stringValue":"{\"host\":\"144.131.190.47\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"PATCH\", \"request\": \"/roi\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22695, \"referer\": \"http://www.centralsynergize.biz/proactive/next-generation\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"502"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770557066772308","observedTimeUnixNano":"1688770557905095435","body":{"stringValue":"{\"host\":\"182.117.211.164\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"PATCH\", \"request\": \"/content/viral\", \"protocol\":\"HTTP/1.0\", \"status\":403, \"bytes\":4228, \"referer\": \"http://www.internalproactive.io/benchmark/extend/roi/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"PATCH"}},{"key":"http_status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557316918605","observedTimeUnixNano":"1688770557905100755","body":{"stringValue":"{\"host\":\"20.32.13.13\", \"user-identifier\":\"yost7451\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"DELETE\", \"request\": \"/transparent\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":20998, \"referer\": \"https://www.productarchitect.com/generate/paradigms\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557566998293","observedTimeUnixNano":"1688770557905104465","body":{"stringValue":"{\"host\":\"47.67.12.210\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:57 +0000\", \"method\": \"GET\", \"request\": \"/vertical/architectures\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":4031, \"referer\": \"https://www.internalcollaborative.com/sexy/enhance/clicks-and-mortar\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557817067841","observedTimeUnixNano":"1688770557905107475","body":{"stringValue":"{\"host\":\"178.160.245.169\", \"user-identifier\":\"stracke6888\", \"datetime\":\"07/Jul/2023:22:55:57 +0000\", \"method\": \"GET\", \"request\": \"/dynamic/24%2f7\", \"protocol\":\"HTTP/2.0\", \"status\":401, \"bytes\":24926, \"referer\": \"http://www.legacye-services.io/e-business/grow/seamless/seize\"}"},"attributes":[{"key":"http_method","value":{"stringValue":"GET"}},{"key":"http_status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556816580510","observedTimeUnixNano":"1688770557905110645","body":{"stringValue":"{\"host\":\"244.189.75.59\", \"user-identifier\":\"jaskolski1628\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"DELETE\", \"request\": \"/revolutionize/whiteboard/e-markets\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":29374, \"referer\": \"http://www.nationalfacilitate.com/real-time\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http_method","value":{"stringValue":"DELETE"}},{"key":"http_status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770531506389037","observedTimeUnixNano":"1688770533485875260","body":{"stringValue":"{\"host\":\"182.232.235.224\", \"user-identifier\":\"murray3567\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"POST\", \"request\": \"/monetize\", \"protocol\":\"HTTP/2.0\", \"status\":503, \"bytes\":536, \"referer\": \"https://www.legacyeyeballs.io/streamline/mindshare/value-added/metrics\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532258117999","observedTimeUnixNano":"1688770533485879030","body":{"stringValue":"{\"host\":\"222.2.138.72\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"PUT\", \"request\": \"/strategize/e-services/24%2f7\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22430, \"referer\": \"https://www.globalapplications.info/scale/scalable/morph/collaborative\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532508421534","observedTimeUnixNano":"1688770533485879850","body":{"stringValue":"{\"host\":\"114.175.148.98\", \"user-identifier\":\"trantow3854\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PUT\", \"request\": \"/killer\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":9706, \"referer\": \"http://www.leadproactive.net/supply-chains\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506099582","observedTimeUnixNano":"1688770533485882410","body":{"stringValue":"{\"host\":\"224.56.79.52\", \"user-identifier\":\"armstrong8550\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"DELETE\", \"request\": \"/applications/incentivize/deliver/24%2f365\", \"protocol\":\"HTTP/1.0\", \"status\":501, \"bytes\":8709, \"referer\": \"http://www.directoptimize.biz/plug-and-play/leverage\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506357477","observedTimeUnixNano":"1688770533485884770","body":{"stringValue":"{\"host\":\"59.254.188.104\", \"user-identifier\":\"shanahan3762\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"PUT\", \"request\": \"/content/productize/expedite/viral\", \"protocol\":\"HTTP/1.0\", \"status\":204, \"bytes\":22262, \"referer\": \"https://www.internalone-to-one.io/productize\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506095582","observedTimeUnixNano":"1688770533485886960","body":{"stringValue":"{\"host\":\"108.166.198.19\", \"user-identifier\":\"rempel2418\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"PUT\", \"request\": \"/infrastructures/dynamic/interfaces\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":12418, \"referer\": \"http://www.directutilize.info/revolutionize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506270749","observedTimeUnixNano":"1688770533485888960","body":{"stringValue":"{\"host\":\"32.172.191.106\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"PATCH\", \"request\": \"/next-generation/efficient\", \"protocol\":\"HTTP/1.0\", \"status\":301, \"bytes\":19187, \"referer\": \"http://www.humanarchitectures.org/content/cross-media/brand/extensible\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"301"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506356337","observedTimeUnixNano":"1688770533485891020","body":{"stringValue":"{\"host\":\"245.62.143.62\", \"user-identifier\":\"kshlerin1534\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"GET\", \"request\": \"/architect/portals/niches\", \"protocol\":\"HTTP/1.0\", \"status\":403, \"bytes\":19870, \"referer\": \"http://www.directtransition.biz/mesh/next-generation/roi\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532007261105","observedTimeUnixNano":"1688770533485893360","body":{"stringValue":"{\"host\":\"112.190.183.173\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"GET\", \"request\": \"/dynamic/sticky\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":21741, \"referer\": \"https://www.humaninfrastructures.biz/users/collaborative/proactive/proactive\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"201"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506066453","observedTimeUnixNano":"1688770533485897459","body":{"stringValue":"{\"host\":\"117.12.231.81\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"PUT\", \"request\": \"/models/empower/open-source\", \"protocol\":\"HTTP/1.1\", \"status\":302, \"bytes\":27722, \"referer\": \"https://www.dynamicmindshare.org/e-business/intuitive/unleash/markets\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506070283","observedTimeUnixNano":"1688770533485899839","body":{"stringValue":"{\"host\":\"236.7.233.166\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"HEAD\", \"request\": \"/enterprise\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":20357, \"referer\": \"https://www.investorseize.net/cultivate/synergies/revolutionize/extend\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506277879","observedTimeUnixNano":"1688770533485900339","body":{"stringValue":"{\"host\":\"55.119.244.237\", \"user-identifier\":\"buckridge5168\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"HEAD\", \"request\": \"/sexy/synthesize/architectures/front-end\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":24908, \"referer\": \"http://www.nationale-markets.net/bricks-and-clicks\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531756857070","observedTimeUnixNano":"1688770533485902439","body":{"stringValue":"{\"host\":\"14.182.107.209\", \"user-identifier\":\"kozey6276\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"DELETE\", \"request\": \"/embrace/cutting-edge/empower/empower\", \"protocol\":\"HTTP/1.0\", \"status\":405, \"bytes\":19380, \"referer\": \"http://www.legacycompelling.name/drive/strategic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506329078","observedTimeUnixNano":"1688770533485904529","body":{"stringValue":"{\"host\":\"80.210.177.228\", \"user-identifier\":\"nitzsche3256\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"GET\", \"request\": \"/paradigms/innovate/b2b/drive\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":923, \"referer\": \"http://www.legacyend-to-end.io/enhance\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506331908","observedTimeUnixNano":"1688770533485906569","body":{"stringValue":"{\"host\":\"86.173.91.15\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"PATCH\", \"request\": \"/open-source/integrate/open-source/visualize\", \"protocol\":\"HTTP/2.0\", \"status\":200, \"bytes\":19653, \"referer\": \"https://www.principalgenerate.io/visualize/dot-com\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506091172","observedTimeUnixNano":"1688770533485908829","body":{"stringValue":"{\"host\":\"214.52.159.245\", \"user-identifier\":\"lowe8277\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"POST\", \"request\": \"/transition/synthesize/applications/24%2f365\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":13965, \"referer\": \"https://www.leadnext-generation.info/redefine/e-business\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506274739","observedTimeUnixNano":"1688770533485910989","body":{"stringValue":"{\"host\":\"79.228.108.32\", \"user-identifier\":\"sanford2465\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"POST\", \"request\": \"/incentivize\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":9074, \"referer\": \"http://www.chiefe-business.info/e-enable/incentivize/transition\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506330508","observedTimeUnixNano":"1688770533485913019","body":{"stringValue":"{\"host\":\"153.126.163.17\", \"user-identifier\":\"terry2422\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"DELETE\", \"request\": \"/e-enable\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":1033, \"referer\": \"http://www.customere-commerce.info/mesh\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506406776","observedTimeUnixNano":"1688770533485915089","body":{"stringValue":"{\"host\":\"106.153.80.249\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"POST\", \"request\": \"/systems\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":26505, \"referer\": \"http://www.chiefsolutions.info/technologies/b2c/embrace/applications\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506063113","observedTimeUnixNano":"1688770533485917269","body":{"stringValue":"{\"host\":\"139.211.170.136\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:26 +0000\", \"method\": \"GET\", \"request\": \"/iterate/implement/relationships\", \"protocol\":\"HTTP/1.1\", \"status\":100, \"bytes\":22426, \"referer\": \"https://www.legacyseamless.io/transparent\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506072323","observedTimeUnixNano":"1688770533485920149","body":{"stringValue":"{\"host\":\"159.238.229.29\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"POST\", \"request\": \"/world-class\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":22475, \"referer\": \"https://www.nationalend-to-end.io/cross-platform/visualize/compelling/virtual\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506267519","observedTimeUnixNano":"1688770533485920629","body":{"stringValue":"{\"host\":\"240.39.214.21\", \"user-identifier\":\"yost2184\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"POST\", \"request\": \"/initiatives/benchmark/mesh/utilize\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":6318, \"referer\": \"http://www.regionalreintermediate.biz/innovative/utilize/morph\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770532759320597","observedTimeUnixNano":"1688770534505475760","body":{"stringValue":"{\"host\":\"219.253.131.24\", \"user-identifier\":\"nolan2452\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"POST\", \"request\": \"/transform/partnerships/compelling\", \"protocol\":\"HTTP/1.0\", \"status\":504, \"bytes\":5432, \"referer\": \"http://www.dynamicpartnerships.info/distributed/e-enable\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"504"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533009722650","observedTimeUnixNano":"1688770534505481330","body":{"stringValue":"{\"host\":\"140.150.10.211\", \"user-identifier\":\"stark5567\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PUT\", \"request\": \"/visionary/harness/exploit\", \"protocol\":\"HTTP/1.0\", \"status\":416, \"bytes\":12222, \"referer\": \"https://www.internalvortals.io/initiatives/niches\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533260610181","observedTimeUnixNano":"1688770534505485390","body":{"stringValue":"{\"host\":\"255.39.188.234\", \"user-identifier\":\"cartwright7210\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PATCH\", \"request\": \"/best-of-breed/transparent\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":28207, \"referer\": \"https://www.productvertical.com/models/harness/bandwidth/scalable\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533511255568","observedTimeUnixNano":"1688770534505488520","body":{"stringValue":"{\"host\":\"32.3.226.175\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"PUT\", \"request\": \"/communities/evolve/content/interfaces\", \"protocol\":\"HTTP/1.0\", \"status\":200, \"bytes\":16759, \"referer\": \"https://www.globalnext-generation.info/seize/engage/unleash/empower\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533761830695","observedTimeUnixNano":"1688770534505491520","body":{"stringValue":"{\"host\":\"224.24.229.231\", \"user-identifier\":\"runte3687\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"POST\", \"request\": \"/e-business/brand/methodologies\", \"protocol\":\"HTTP/2.0\", \"status\":416, \"bytes\":13103, \"referer\": \"https://www.regionalrich.com/action-items\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534012191357","observedTimeUnixNano":"1688770534505494330","body":{"stringValue":"{\"host\":\"34.60.133.32\", \"user-identifier\":\"wisoky3066\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"PUT\", \"request\": \"/out-of-the-box\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":5665, \"referer\": \"http://www.leadrevolutionize.org/efficient/partnerships/architect/solutions\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534263072946","observedTimeUnixNano":"1688770534505497230","body":{"stringValue":"{\"host\":\"138.73.65.106\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"HEAD\", \"request\": \"/deploy/front-end/vertical/metrics\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":28841, \"referer\": \"https://www.regionalsyndicate.name/solutions\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770535014595986","observedTimeUnixNano":"1688770535605373502","body":{"stringValue":"{\"host\":\"119.217.207.214\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"POST\", \"request\": \"/magnetic/reinvent/virtual\", \"protocol\":\"HTTP/2.0\", \"status\":204, \"bytes\":23155, \"referer\": \"http://www.leadbandwidth.com/brand/unleash\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770535265453194","observedTimeUnixNano":"1688770535605378152","body":{"stringValue":"{\"host\":\"20.82.163.90\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"GET\", \"request\": \"/seamless/seamless/recontextualize\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":7620, \"referer\": \"http://www.leaduser-centric.io/reinvent/engineer/experiences\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770535515889000","observedTimeUnixNano":"1688770535605381162","body":{"stringValue":"{\"host\":\"75.110.63.232\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PUT\", \"request\": \"/one-to-one/aggregate\", \"protocol\":\"HTTP/1.1\", \"status\":416, \"bytes\":21091, \"referer\": \"https://www.regionalimplement.name/relationships/engage/e-services/redefine\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534514124542","observedTimeUnixNano":"1688770535605383892","body":{"stringValue":"{\"host\":\"150.204.109.167\", \"user-identifier\":\"swift5175\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"HEAD\", \"request\": \"/integrated/distributed/methodologies\", \"protocol\":\"HTTP/2.0\", \"status\":404, \"bytes\":10857, \"referer\": \"https://www.seniorsexy.com/distributed\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534764273486","observedTimeUnixNano":"1688770535605386732","body":{"stringValue":"{\"host\":\"15.176.190.242\", \"user-identifier\":\"champlin5148\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"PUT\", \"request\": \"/monetize/mindshare/rich/extensible\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":17396, \"referer\": \"http://www.globalrepurpose.com/redefine/brand/architect\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770535766768587","observedTimeUnixNano":"1688770536805705482","body":{"stringValue":"{\"host\":\"159.240.99.65\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"GET\", \"request\": \"/seamless\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":4573, \"referer\": \"https://www.forwardgenerate.org/revolutionize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536017199113","observedTimeUnixNano":"1688770536805710712","body":{"stringValue":"{\"host\":\"254.246.88.15\", \"user-identifier\":\"feest4001\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PUT\", \"request\": \"/architectures\", \"protocol\":\"HTTP/1.1\", \"status\":504, \"bytes\":9703, \"referer\": \"http://www.seniorgenerate.name/roi/web-enabled/mesh\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"504"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536267969591","observedTimeUnixNano":"1688770536805714342","body":{"stringValue":"{\"host\":\"117.113.156.145\", \"user-identifier\":\"hansen6831\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PATCH\", \"request\": \"/best-of-breed/strategize/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":500, \"bytes\":7590, \"referer\": \"http://www.internalnext-generation.biz/granular/relationships\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536518702679","observedTimeUnixNano":"1688770536805717532","body":{"stringValue":"{\"host\":\"108.138.59.166\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"GET\", \"request\": \"/proactive\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":20470, \"referer\": \"https://www.corporatevisionary.org/models/integrate/supply-chains/visualize\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"502"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536769226151","observedTimeUnixNano":"1688770536805720572","body":{"stringValue":"{\"host\":\"80.21.65.147\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"POST\", \"request\": \"/methodologies\", \"protocol\":\"HTTP/1.1\", \"status\":504, \"bytes\":24809, \"referer\": \"https://www.leade-tailers.io/front-end/turn-key/supply-chains\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"504"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770537019590646","observedTimeUnixNano":"1688770538105581695","body":{"stringValue":"{\"host\":\"48.48.132.74\", \"user-identifier\":\"sauer4564\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"PATCH\", \"request\": \"/ubiquitous/transparent\", \"protocol\":\"HTTP/1.1\", \"status\":204, \"bytes\":21351, \"referer\": \"http://www.corporatebrand.com/synergies\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"204"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537270539598","observedTimeUnixNano":"1688770538105586155","body":{"stringValue":"{\"host\":\"41.84.80.177\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"POST\", \"request\": \"/schemas\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":9416, \"referer\": \"http://www.forwardrich.name/bleeding-edge\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537520765634","observedTimeUnixNano":"1688770538105589205","body":{"stringValue":"{\"host\":\"215.39.102.24\", \"user-identifier\":\"pouros8556\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"HEAD\", \"request\": \"/infrastructures\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":3738, \"referer\": \"https://www.internationalrevolutionary.io/incentivize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537771653837","observedTimeUnixNano":"1688770538105592045","body":{"stringValue":"{\"host\":\"233.20.118.85\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"PATCH\", \"request\": \"/bandwidth/whiteboard/relationships/b2b\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":13768, \"referer\": \"https://www.internalchannels.io/synthesize/customized/solutions/target\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538022089288","observedTimeUnixNano":"1688770538105594684","body":{"stringValue":"{\"host\":\"76.247.19.239\", \"user-identifier\":\"cormier5083\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"DELETE\", \"request\": \"/interfaces\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":4639, \"referer\": \"https://www.principalengineer.biz/solutions/synergistic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770538272986089","observedTimeUnixNano":"1688770539304866280","body":{"stringValue":"{\"host\":\"182.27.181.198\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"HEAD\", \"request\": \"/deliver/e-tailers\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":13889, \"referer\": \"http://www.investorextensible.com/empower/clicks-and-mortar/dot-com\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538523374810","observedTimeUnixNano":"1688770539304871430","body":{"stringValue":"{\"host\":\"135.212.14.226\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/integrate/e-business/orchestrate\", \"protocol\":\"HTTP/1.1\", \"status\":416, \"bytes\":25334, \"referer\": \"http://www.investordynamic.name/benchmark/mission-critical\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538774176503","observedTimeUnixNano":"1688770539304874460","body":{"stringValue":"{\"host\":\"47.20.233.211\", \"user-identifier\":\"schowalter7876\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/bricks-and-clicks/channels\", \"protocol\":\"HTTP/2.0\", \"status\":204, \"bytes\":11038, \"referer\": \"http://www.dynamicmission-critical.info/eyeballs\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539275422243","observedTimeUnixNano":"1688770539304875030","body":{"stringValue":"{\"host\":\"251.240.185.7\", \"user-identifier\":\"kirlin6527\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/embrace/next-generation/value-added\", \"protocol\":\"HTTP/1.0\", \"status\":204, \"bytes\":2768, \"referer\": \"http://www.leadembrace.io/brand\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539024541514","observedTimeUnixNano":"1688770539304877770","body":{"stringValue":"{\"host\":\"225.112.239.87\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"PATCH\", \"request\": \"/vertical/customized\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":25553, \"referer\": \"https://www.districtenterprise.net/schemas/drive\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"500"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770539525776242","observedTimeUnixNano":"1688770540604713526","body":{"stringValue":"{\"host\":\"110.213.172.140\", \"user-identifier\":\"flatley2072\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"POST\", \"request\": \"/back-end/plug-and-play\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":1079, \"referer\": \"https://www.corporatesyndicate.net/proactive/end-to-end/out-of-the-box/partnerships\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539776638361","observedTimeUnixNano":"1688770540604718996","body":{"stringValue":"{\"host\":\"132.59.120.128\", \"user-identifier\":\"kautzer2610\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"GET\", \"request\": \"/embrace/mission-critical\", \"protocol\":\"HTTP/2.0\", \"status\":403, \"bytes\":9174, \"referer\": \"https://www.humanengineer.name/engineer/value-added/interactive\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540027065299","observedTimeUnixNano":"1688770540604723106","body":{"stringValue":"{\"host\":\"255.128.41.196\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"GET\", \"request\": \"/dynamic\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":23462, \"referer\": \"http://www.seniorenterprise.net/best-of-breed/grow/cross-media\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540277926986","observedTimeUnixNano":"1688770540604726566","body":{"stringValue":"{\"host\":\"108.113.41.211\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"HEAD\", \"request\": \"/cutting-edge/deliverables/cross-media\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":18274, \"referer\": \"https://www.directvisualize.io/vortals/24/7\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540528287773","observedTimeUnixNano":"1688770540604729666","body":{"stringValue":"{\"host\":\"126.219.241.28\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"GET\", \"request\": \"/expedite/engineer\", \"protocol\":\"HTTP/2.0\", \"status\":503, \"bytes\":9425, \"referer\": \"http://www.internalrecontextualize.info/content/facilitate\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770540779141751","observedTimeUnixNano":"1688770541805060126","body":{"stringValue":"{\"host\":\"75.100.204.75\", \"user-identifier\":\"stiedemann6660\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"PATCH\", \"request\": \"/incubate/syndicate/communities/enhance\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":28778, \"referer\": \"https://www.principalmarkets.biz/web-readiness/holistic\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541029484568","observedTimeUnixNano":"1688770541805067016","body":{"stringValue":"{\"host\":\"164.61.170.176\", \"user-identifier\":\"feil3053\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"PUT\", \"request\": \"/deliver\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":12408, \"referer\": \"https://www.seniordot-com.net/best-of-breed/wireless/partnerships/magnetic\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"500"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541280380253","observedTimeUnixNano":"1688770541805070706","body":{"stringValue":"{\"host\":\"237.229.232.170\", \"user-identifier\":\"cummings3317\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"HEAD\", \"request\": \"/technologies\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":23099, \"referer\": \"https://www.nationalsynergistic.net/cross-media/frictionless\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541531131440","observedTimeUnixNano":"1688770541805073826","body":{"stringValue":"{\"host\":\"191.42.120.4\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"DELETE\", \"request\": \"/cultivate/enterprise\", \"protocol\":\"HTTP/1.1\", \"status\":503, \"bytes\":1096, \"referer\": \"http://www.regionalend-to-end.com/e-commerce/bricks-and-clicks\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541781600583","observedTimeUnixNano":"1688770541805077586","body":{"stringValue":"{\"host\":\"115.72.122.202\", \"user-identifier\":\"swift7514\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"PATCH\", \"request\": \"/transition\", \"protocol\":\"HTTP/2.0\", \"status\":405, \"bytes\":2870, \"referer\": \"https://www.leadenvisioneer.biz/compelling/killer\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770542031960818","observedTimeUnixNano":"1688770543105018384","body":{"stringValue":"{\"host\":\"125.249.12.227\", \"user-identifier\":\"crona1635\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"GET\", \"request\": \"/schemas\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":23913, \"referer\": \"https://www.leadinteractive.name/synthesize/deliverables/unleash/viral\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542784196884","observedTimeUnixNano":"1688770543105020304","body":{"stringValue":"{\"host\":\"31.184.47.132\", \"user-identifier\":\"boyer7325\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"GET\", \"request\": \"/24%2f7/web+services/extensible/reintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":17233, \"referer\": \"https://www.leadredefine.com/morph/exploit/seize/whiteboard\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542282846871","observedTimeUnixNano":"1688770543105023974","body":{"stringValue":"{\"host\":\"180.227.124.253\", \"user-identifier\":\"glover2525\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"HEAD\", \"request\": \"/roi/revolutionary\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":4997, \"referer\": \"https://www.investorrecontextualize.net/cross-platform/syndicate/initiatives/content\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542533294682","observedTimeUnixNano":"1688770543105026754","body":{"stringValue":"{\"host\":\"35.167.110.144\", \"user-identifier\":\"kiehn6344\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"PATCH\", \"request\": \"/user-centric/incubate/disintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":503, \"bytes\":19773, \"referer\": \"https://www.investorvertical.info/drive/streamline/e-services/reinvent\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543034516918","observedTimeUnixNano":"1688770543105029564","body":{"stringValue":"{\"host\":\"135.76.24.123\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"GET\", \"request\": \"/infrastructures\", \"protocol\":\"HTTP/1.1\", \"status\":501, \"bytes\":29365, \"referer\": \"http://www.dynamicweb-enabled.com/web-readiness/markets/revolutionize\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770543285385529","observedTimeUnixNano":"1688770544305313540","body":{"stringValue":"{\"host\":\"204.141.204.194\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"DELETE\", \"request\": \"/rich\", \"protocol\":\"HTTP/1.1\", \"status\":401, \"bytes\":6762, \"referer\": \"https://www.principaldeliver.biz/reintermediate/channels/impactful\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543535766279","observedTimeUnixNano":"1688770544305318969","body":{"stringValue":"{\"host\":\"25.50.100.83\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"DELETE\", \"request\": \"/sexy/web-readiness/infomediaries\", \"protocol\":\"HTTP/2.0\", \"status\":400, \"bytes\":8067, \"referer\": \"https://www.customermethodologies.com/user-centric/schemas/deliverables/synergize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543786551452","observedTimeUnixNano":"1688770544305322659","body":{"stringValue":"{\"host\":\"172.52.222.78\", \"user-identifier\":\"waelchi8475\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"PUT\", \"request\": \"/leverage/syndicate\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":13351, \"referer\": \"http://www.dynamicweb-readiness.net/vertical/holistic/magnetic/content\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544037029250","observedTimeUnixNano":"1688770544305326179","body":{"stringValue":"{\"host\":\"18.33.245.88\", \"user-identifier\":\"brekke5341\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"HEAD\", \"request\": \"/world-class\", \"protocol\":\"HTTP/1.1\", \"status\":100, \"bytes\":23594, \"referer\": \"https://www.districtrevolutionary.org/brand\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544287848500","observedTimeUnixNano":"1688770544305331319","body":{"stringValue":"{\"host\":\"95.146.146.56\", \"user-identifier\":\"witting8863\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"POST\", \"request\": \"/content\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22835, \"referer\": \"https://www.internationalb2c.org/recontextualize/recontextualize/grow/transform\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"502"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770545540651551","observedTimeUnixNano":"1688770545605162902","body":{"stringValue":"{\"host\":\"72.242.71.239\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"POST\", \"request\": \"/global\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":3026, \"referer\": \"http://www.nationalinfomediaries.net/solutions/roi/innovative\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544538215259","observedTimeUnixNano":"1688770545605168132","body":{"stringValue":"{\"host\":\"221.31.17.132\", \"user-identifier\":\"hammes8155\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"PUT\", \"request\": \"/innovate\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":18182, \"referer\": \"http://www.districtempower.name/architectures\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544789083788","observedTimeUnixNano":"1688770545605171772","body":{"stringValue":"{\"host\":\"73.25.185.22\", \"user-identifier\":\"schaefer3552\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"HEAD\", \"request\": \"/collaborative/vertical\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":13514, \"referer\": \"https://www.legacyproductize.net/interfaces\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545039444047","observedTimeUnixNano":"1688770545605175002","body":{"stringValue":"{\"host\":\"133.159.101.254\", \"user-identifier\":\"bernier7784\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"POST\", \"request\": \"/deploy/synergize\", \"protocol\":\"HTTP/1.1\", \"status\":405, \"bytes\":4079, \"referer\": \"http://www.internalinterfaces.io/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545290299404","observedTimeUnixNano":"1688770545605178022","body":{"stringValue":"{\"host\":\"63.166.3.113\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"PUT\", \"request\": \"/networks/e-enable\", \"protocol\":\"HTTP/2.0\", \"status\":301, \"bytes\":23413, \"referer\": \"http://www.leadcontent.info/portals/content/transparent\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"301"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770546292750429","observedTimeUnixNano":"1688770546805390176","body":{"stringValue":"{\"host\":\"49.154.204.143\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"PATCH\", \"request\": \"/solutions\", \"protocol\":\"HTTP/2.0\", \"status\":405, \"bytes\":12772, \"referer\": \"https://www.chieftechnologies.com/deliver\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546543451957","observedTimeUnixNano":"1688770546805394616","body":{"stringValue":"{\"host\":\"139.113.227.72\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"PATCH\", \"request\": \"/users/dynamic/revolutionary\", \"protocol\":\"HTTP/1.1\", \"status\":203, \"bytes\":11927, \"referer\": \"http://www.investorsynergies.org/leading-edge/productize/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546793929399","observedTimeUnixNano":"1688770546805397786","body":{"stringValue":"{\"host\":\"18.225.144.149\", \"user-identifier\":\"macejkovic4811\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"DELETE\", \"request\": \"/cross-platform/optimize/e-tailers\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":8359, \"referer\": \"https://www.districtmesh.name/channels/partnerships/utilize/plug-and-play\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545791539778","observedTimeUnixNano":"1688770546805401576","body":{"stringValue":"{\"host\":\"113.112.205.16\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"DELETE\", \"request\": \"/generate/schemas/impactful/frictionless\", \"protocol\":\"HTTP/1.0\", \"status\":501, \"bytes\":2572, \"referer\": \"https://www.chiefiterate.info/unleash\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"501"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546041916043","observedTimeUnixNano":"1688770546805404055","body":{"stringValue":"{\"host\":\"147.251.113.164\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"GET\", \"request\": \"/integrated\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":10412, \"referer\": \"http://www.productrobust.io/strategic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770547044430211","observedTimeUnixNano":"1688770548105337580","body":{"stringValue":"{\"host\":\"31.12.63.70\", \"user-identifier\":\"carroll3128\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"POST\", \"request\": \"/value-added/infomediaries\", \"protocol\":\"HTTP/1.1\", \"status\":201, \"bytes\":15247, \"referer\": \"https://www.principalopen-source.name/turn-key/facilitate/orchestrate\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547295243595","observedTimeUnixNano":"1688770548105342880","body":{"stringValue":"{\"host\":\"160.136.74.178\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"PUT\", \"request\": \"/productize/reintermediate/markets\", \"protocol\":\"HTTP/1.0\", \"status\":100, \"bytes\":28739, \"referer\": \"http://www.seniorplatforms.name/infrastructures\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"100"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547545668186","observedTimeUnixNano":"1688770548105346740","body":{"stringValue":"{\"host\":\"218.137.103.90\", \"user-identifier\":\"huels6041\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"POST\", \"request\": \"/enhance\", \"protocol\":\"HTTP/1.0\", \"status\":401, \"bytes\":1115, \"referer\": \"https://www.internalrevolutionary.com/roi/productize\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547796500899","observedTimeUnixNano":"1688770548105350040","body":{"stringValue":"{\"host\":\"152.7.8.228\", \"user-identifier\":\"mosciski3102\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"PUT\", \"request\": \"/matrix/content\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":9350, \"referer\": \"http://www.humanreintermediate.com/architectures/web-enabled/24/365\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548046805313","observedTimeUnixNano":"1688770548105353230","body":{"stringValue":"{\"host\":\"189.145.183.88\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"HEAD\", \"request\": \"/mesh/b2b/syndicate/technologies\", \"protocol\":\"HTTP/1.1\", \"status\":200, \"bytes\":23769, \"referer\": \"http://www.humanparadigms.info/world-class\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770548548089763","observedTimeUnixNano":"1688770549305574358","body":{"stringValue":"{\"host\":\"149.219.8.124\", \"user-identifier\":\"johnston5032\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"DELETE\", \"request\": \"/convergence\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":20805, \"referer\": \"http://www.dynamicuser-centric.com/schemas/sexy/granular/bricks-and-clicks\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548798941415","observedTimeUnixNano":"1688770549305580998","body":{"stringValue":"{\"host\":\"202.159.240.173\", \"user-identifier\":\"hayes4575\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"POST\", \"request\": \"/cross-platform/content/functionalities/repurpose\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":21455, \"referer\": \"http://www.regionalgenerate.info/b2c/vortals/solutions\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549049313785","observedTimeUnixNano":"1688770549305588108","body":{"stringValue":"{\"host\":\"120.119.68.91\", \"user-identifier\":\"heidenreich7556\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"HEAD\", \"request\": \"/aggregate/networks\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":26555, \"referer\": \"http://www.dynamicvisionary.name/scalable\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549300083475","observedTimeUnixNano":"1688770549305592998","body":{"stringValue":"{\"host\":\"23.194.158.90\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"PATCH\", \"request\": \"/revolutionize/envisioneer/supply-chains\", \"protocol\":\"HTTP/1.1\", \"status\":302, \"bytes\":24515, \"referer\": \"https://www.dynamicproactive.io/customized/deliverables\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548297678124","observedTimeUnixNano":"1688770549305597048","body":{"stringValue":"{\"host\":\"171.235.211.64\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"PUT\", \"request\": \"/proactive/expedite/seize/leading-edge\", \"protocol\":\"HTTP/1.1\", \"status\":403, \"bytes\":16631, \"referer\": \"https://www.humanengineer.info/extend\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770549550496684","observedTimeUnixNano":"1688770550605255542","body":{"stringValue":"{\"host\":\"35.243.14.54\", \"user-identifier\":\"weimann5622\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"DELETE\", \"request\": \"/seamless\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":27406, \"referer\": \"https://www.regionalembrace.biz/e-business/24/7\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550552786577","observedTimeUnixNano":"1688770550605257182","body":{"stringValue":"{\"host\":\"206.39.201.209\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"DELETE\", \"request\": \"/transform/infrastructures\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":17105, \"referer\": \"https://www.leadsticky.info/wireless/one-to-one\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549801322883","observedTimeUnixNano":"1688770550605263581","body":{"stringValue":"{\"host\":\"101.240.9.159\", \"user-identifier\":\"turcotte5453\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"GET\", \"request\": \"/revolutionary/revolutionize/incubate/reintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":10229, \"referer\": \"http://www.forwardnetworks.org/generate/exploit/b2c/web services\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"406"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550051625492","observedTimeUnixNano":"1688770550605268311","body":{"stringValue":"{\"host\":\"148.16.104.181\", \"user-identifier\":\"bednar5574\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"GET\", \"request\": \"/collaborative/seize\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":13672, \"referer\": \"https://www.internationalinfrastructures.io/optimize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550302524748","observedTimeUnixNano":"1688770550605271551","body":{"stringValue":"{\"host\":\"12.50.238.58\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"DELETE\", \"request\": \"/value-added/supply-chains\", \"protocol\":\"HTTP/1.0\", \"status\":200, \"bytes\":19948, \"referer\": \"http://www.investorefficient.biz/intuitive/24/7\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"200"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770550803736612","observedTimeUnixNano":"1688770551805598004","body":{"stringValue":"{\"host\":\"6.57.16.190\", \"user-identifier\":\"koelpin1807\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"HEAD\", \"request\": \"/mesh\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":10734, \"referer\": \"http://www.regionalbandwidth.com/integrate/benchmark\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551054058149","observedTimeUnixNano":"1688770551805607224","body":{"stringValue":"{\"host\":\"77.236.31.140\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"PATCH\", \"request\": \"/mission-critical\", \"protocol\":\"HTTP/2.0\", \"status\":201, \"bytes\":22561, \"referer\": \"https://www.directmaximize.com/supply-chains/deliverables/web-enabled\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551305051212","observedTimeUnixNano":"1688770551805613764","body":{"stringValue":"{\"host\":\"96.170.101.152\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"GET\", \"request\": \"/implement/cultivate/revolutionary\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":19505, \"referer\": \"http://www.directrobust.org/dot-com/benchmark\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551555515704","observedTimeUnixNano":"1688770551805620524","body":{"stringValue":"{\"host\":\"250.164.115.76\", \"user-identifier\":\"hermiston4072\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PUT\", \"request\": \"/experiences\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":18299, \"referer\": \"http://www.directrobust.name/initiatives\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770551806068175","observedTimeUnixNano":"1688770552904845153","body":{"stringValue":"{\"host\":\"254.168.97.108\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PUT\", \"request\": \"/vertical/robust/incubate\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":12490, \"referer\": \"http://www.productweb services.name/redefine/engage/bleeding-edge/maximize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552056479008","observedTimeUnixNano":"1688770552904851803","body":{"stringValue":"{\"host\":\"118.204.193.76\", \"user-identifier\":\"kihn1514\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"DELETE\", \"request\": \"/dot-com/scalable/interfaces\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":1951, \"referer\": \"https://www.dynamicleading-edge.org/customized/deliverables\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"502"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552808876801","observedTimeUnixNano":"1688770552904853303","body":{"stringValue":"{\"host\":\"239.84.230.236\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"DELETE\", \"request\": \"/back-end/target\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":13592, \"referer\": \"http://www.corporatefacilitate.io/orchestrate/global\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552306744053","observedTimeUnixNano":"1688770552904857773","body":{"stringValue":"{\"host\":\"11.118.181.255\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PATCH\", \"request\": \"/one-to-one/channels/networks/collaborative\", \"protocol\":\"HTTP/1.1\", \"status\":501, \"bytes\":25565, \"referer\": \"http://www.dynamicinterfaces.biz/deliverables/real-time/global\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552557804932","observedTimeUnixNano":"1688770552904866042","body":{"stringValue":"{\"host\":\"38.157.64.60\", \"user-identifier\":\"collins3161\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"POST\", \"request\": \"/leverage/convergence\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":7451, \"referer\": \"http://www.dynamicapplications.name/scalable\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770553059904090","observedTimeUnixNano":"1688770554104914869","body":{"stringValue":"{\"host\":\"91.210.58.64\", \"user-identifier\":\"hirthe5885\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"POST\", \"request\": \"/experiences\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":26372, \"referer\": \"https://www.globalreintermediate.biz/scalable/strategic/deliver\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553310125793","observedTimeUnixNano":"1688770554104921949","body":{"stringValue":"{\"host\":\"171.146.184.207\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"HEAD\", \"request\": \"/global/wireless/initiatives\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":3559, \"referer\": \"http://www.districtextensible.info/sexy\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553561204179","observedTimeUnixNano":"1688770554104926178","body":{"stringValue":"{\"host\":\"223.135.210.193\", \"user-identifier\":\"lakin5175\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"DELETE\", \"request\": \"/e-commerce\", \"protocol\":\"HTTP/1.1\", \"status\":301, \"bytes\":17589, \"referer\": \"http://www.investortransform.org/embrace\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"301"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553811251136","observedTimeUnixNano":"1688770554104929478","body":{"stringValue":"{\"host\":\"167.99.122.25\", \"user-identifier\":\"dietrich4410\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"DELETE\", \"request\": \"/visualize/impactful/bleeding-edge/portals\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":21701, \"referer\": \"http://www.seniorredefine.info/e-commerce\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554061484609","observedTimeUnixNano":"1688770554104932588","body":{"stringValue":"{\"host\":\"12.8.67.46\", \"user-identifier\":\"stracke8218\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"HEAD\", \"request\": \"/ubiquitous/morph/visualize\", \"protocol\":\"HTTP/1.0\", \"status\":401, \"bytes\":3223, \"referer\": \"http://www.regionaldisintermediate.name/communities/synergies/drive\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770555062845794","observedTimeUnixNano":"1688770555404999753","body":{"stringValue":"{\"host\":\"68.65.52.180\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"PATCH\", \"request\": \"/engage/web-enabled/virtual/metrics\", \"protocol\":\"HTTP/2.0\", \"status\":100, \"bytes\":22935, \"referer\": \"http://www.dynamicfront-end.info/revolutionize/architect\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554311549923","observedTimeUnixNano":"1688770555405006503","body":{"stringValue":"{\"host\":\"58.176.246.52\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"HEAD\", \"request\": \"/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":26702, \"referer\": \"http://www.corporateniches.biz/exploit\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"400"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770555313060774","observedTimeUnixNano":"1688770555405007793","body":{"stringValue":"{\"host\":\"219.158.106.54\", \"user-identifier\":\"treutel4445\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"HEAD\", \"request\": \"/granular/killer/enhance/leverage\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":7060, \"referer\": \"http://www.productmonetize.org/visualize/paradigms/orchestrate/seize\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554561762855","observedTimeUnixNano":"1688770555405012603","body":{"stringValue":"{\"host\":\"89.109.67.122\", \"user-identifier\":\"lang1745\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"PATCH\", \"request\": \"/syndicate\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":4309, \"referer\": \"http://www.chiefintuitive.biz/sexy/generate/end-to-end\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554812778300","observedTimeUnixNano":"1688770555405016873","body":{"stringValue":"{\"host\":\"108.149.26.184\", \"user-identifier\":\"wyman8181\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"DELETE\", \"request\": \"/infrastructures/utilize/mindshare\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":8395, \"referer\": \"https://www.dynamicintuitive.biz/efficient/eyeballs/seamless/enable\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770555564076188","observedTimeUnixNano":"1688770556605475977","body":{"stringValue":"{\"host\":\"214.112.255.154\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"POST\", \"request\": \"/metrics/24%2f7/deliverables/technologies\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":8201, \"referer\": \"http://www.centralfacilitate.net/convergence/repurpose\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770555814316806","observedTimeUnixNano":"1688770556605481157","body":{"stringValue":"{\"host\":\"222.210.64.103\", \"user-identifier\":\"jaskolski1751\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"HEAD\", \"request\": \"/synergistic\", \"protocol\":\"HTTP/2.0\", \"status\":500, \"bytes\":1213, \"referer\": \"http://www.internationalbenchmark.info/cross-platform/infomediaries\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556065384879","observedTimeUnixNano":"1688770556605484527","body":{"stringValue":"{\"host\":\"86.90.114.24\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"GET\", \"request\": \"/synergize/incubate/platforms\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":8681, \"referer\": \"http://www.forwardvisualize.info/technologies/holistic\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556315409620","observedTimeUnixNano":"1688770556605487627","body":{"stringValue":"{\"host\":\"234.206.18.83\", \"user-identifier\":\"kozey6774\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"GET\", \"request\": \"/synthesize/users/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":500, \"bytes\":24463, \"referer\": \"http://www.legacyplug-and-play.net/mesh\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556566005240","observedTimeUnixNano":"1688770556605490717","body":{"stringValue":"{\"host\":\"144.131.190.47\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"PATCH\", \"request\": \"/roi\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22695, \"referer\": \"http://www.centralsynergize.biz/proactive/next-generation\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"502"}}],"traceId":"","spanId":""}]}]}]} +{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770557066772308","observedTimeUnixNano":"1688770557905095435","body":{"stringValue":"{\"host\":\"182.117.211.164\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"PATCH\", \"request\": \"/content/viral\", \"protocol\":\"HTTP/1.0\", \"status\":403, \"bytes\":4228, \"referer\": \"http://www.internalproactive.io/benchmark/extend/roi/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557316918605","observedTimeUnixNano":"1688770557905100755","body":{"stringValue":"{\"host\":\"20.32.13.13\", \"user-identifier\":\"yost7451\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"DELETE\", \"request\": \"/transparent\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":20998, \"referer\": \"https://www.productarchitect.com/generate/paradigms\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557566998293","observedTimeUnixNano":"1688770557905104465","body":{"stringValue":"{\"host\":\"47.67.12.210\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:57 +0000\", \"method\": \"GET\", \"request\": \"/vertical/architectures\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":4031, \"referer\": \"https://www.internalcollaborative.com/sexy/enhance/clicks-and-mortar\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557817067841","observedTimeUnixNano":"1688770557905107475","body":{"stringValue":"{\"host\":\"178.160.245.169\", \"user-identifier\":\"stracke6888\", \"datetime\":\"07/Jul/2023:22:55:57 +0000\", \"method\": \"GET\", \"request\": \"/dynamic/24%2f7\", \"protocol\":\"HTTP/2.0\", \"status\":401, \"bytes\":24926, \"referer\": \"http://www.legacye-services.io/e-business/grow/seamless/seize\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556816580510","observedTimeUnixNano":"1688770557905110645","body":{"stringValue":"{\"host\":\"244.189.75.59\", \"user-identifier\":\"jaskolski1628\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"DELETE\", \"request\": \"/revolutionize/whiteboard/e-markets\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":29374, \"referer\": \"http://www.nationalfacilitate.com/real-time\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} diff --git a/integration/lokie2e/ch_test.go b/integration/lokie2e/ch_test.go index b7af8dfa..e8cf3a58 100644 --- a/integration/lokie2e/ch_test.go +++ b/integration/lokie2e/ch_test.go @@ -2,7 +2,8 @@ package lokie2e_test import ( "context" - "strings" + "crypto/rand" + "fmt" "testing" "time" @@ -11,7 +12,6 @@ import ( "github.com/cenkalti/backoff/v4" "github.com/go-faster/errors" "github.com/go-faster/sdk/zctx" - "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/testcontainers/testcontainers-go" "go.uber.org/zap/zaptest" @@ -20,6 +20,12 @@ import ( "github.com/go-faster/oteldb/internal/chstorage" ) +func randomPrefix() string { + var data [6]byte + _, _ = rand.Read(data[:]) + return fmt.Sprintf("%x", data[:]) +} + func TestCH(t *testing.T) { integration.Skip(t) ctx := context.Background() @@ -61,7 +67,7 @@ func TestCH(t *testing.T) { t.Fatal(err) } - prefix := strings.ReplaceAll(uuid.NewString(), "-", "") + prefix := randomPrefix() tables := chstorage.DefaultTables() tables.Each(func(name *string) error { old := *name diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index c54735a0..2ed51bcb 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -130,69 +130,69 @@ func runTest( }{ // Label matchers. // Effectively match GET. - {`{http_method="GET"}`, 21}, - {`{http_method=~".*GET.*"}`, 21}, - {`{http_method=~"^GET$"}`, 21}, - {`{http_method!~"(HEAD|POST|DELETE|PUT|PATCH|TRACE|OPTIONS)"}`, 21}, + {`{http.method="GET"}`, 21}, + {`{http.method=~".*GET.*"}`, 21}, + {`{http.method=~"^GET$"}`, 21}, + {`{http.method!~"(HEAD|POST|DELETE|PUT|PATCH|TRACE|OPTIONS)"}`, 21}, // Try other methods. - {`{http_method="DELETE"}`, 20}, - {`{http_method="GET"}`, 21}, - {`{http_method="HEAD"}`, 22}, - {`{http_method="PATCH"}`, 19}, - {`{http_method="POST"}`, 21}, - {`{http_method="PUT"}`, 20}, - {`{http_method="GET"} | json`, 21}, + {`{http.method="DELETE"}`, 20}, + {`{http.method="GET"}`, 21}, + {`{http.method="HEAD"}`, 22}, + {`{http.method="PATCH"}`, 19}, + {`{http.method="POST"}`, 21}, + {`{http.method="PUT"}`, 20}, + {`{http.method="GET"} | json`, 21}, // Negative label matcher. - {`{http_method!="HEAD"}`, len(set.Records) - 22}, - {`{http_method!~"^HEAD$"}`, len(set.Records) - 22}, + {`{http.method!="HEAD"}`, len(set.Records) - 22}, + {`{http.method!~"^HEAD$"}`, len(set.Records) - 22}, // Multiple lables. - {`{http_method="HEAD",http_status="500"}`, 2}, - {`{http_method="HEAD",http_status=~"^500$"}`, 2}, - {`{http_method=~".*HEAD.*",http_status=~"^500$"}`, 2}, + {`{http.method="HEAD",http.status="500"}`, 2}, + {`{http.method="HEAD",http.status=~"^500$"}`, 2}, + {`{http.method=~".*HEAD.*",http.status=~"^500$"}`, 2}, // Line filter. - {`{http_method=~".+"} |= "\"method\": \"GET\""`, 21}, - {`{http_method=~".+"} |= "\"method\": \"DELETE\""`, 20}, - {`{http_method=~".+"} |= "\"method\": \"HEAD\"" |= "\"status\":500"`, 2}, - {`{http_method=~".+"} |~ "\"method\":\\s*\"DELETE\""`, 20}, - {`{http_method=~".+"} |~ "\"method\":\\s*\"HEAD\"" |= "\"status\":500"`, 2}, + {`{http.method=~".+"} |= "\"method\": \"GET\""`, 21}, + {`{http.method=~".+"} |= "\"method\": \"DELETE\""`, 20}, + {`{http.method=~".+"} |= "\"method\": \"HEAD\"" |= "\"status\":500"`, 2}, + {`{http.method=~".+"} |~ "\"method\":\\s*\"DELETE\""`, 20}, + {`{http.method=~".+"} |~ "\"method\":\\s*\"HEAD\"" |= "\"status\":500"`, 2}, // Try to not use offloading. - {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "\"method\": \"DELETE\""`, 20}, - {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "\"method\": \"HEAD\"" |= "\"status\":500"`, 2}, - {`{http_method=~".+"} |= "\"method\": \"HEAD\"" | line_format "{{ __line__ }}" |= "\"status\":500"`, 2}, + {`{http.method=~".+"} | line_format "{{ __line__ }}" |= "\"method\": \"DELETE\""`, 20}, + {`{http.method=~".+"} | line_format "{{ __line__ }}" |= "\"method\": \"HEAD\"" |= "\"status\":500"`, 2}, + {`{http.method=~".+"} |= "\"method\": \"HEAD\"" | line_format "{{ __line__ }}" |= "\"status\":500"`, 2}, // Negative line matcher. - {`{http_method=~".+"} != "\"method\": \"HEAD\""`, len(set.Records) - 22}, - {`{http_method=~".+"} !~ "\"method\":\\s*\"HEAD\""`, len(set.Records) - 22}, + {`{http.method=~".+"} != "\"method\": \"HEAD\""`, len(set.Records) - 22}, + {`{http.method=~".+"} !~ "\"method\":\\s*\"HEAD\""`, len(set.Records) - 22}, // IP line filter. - {`{http_method="HEAD"} |= ip("236.7.233.166")`, 1}, + {`{http.method="HEAD"} |= ip("236.7.233.166")`, 1}, // Label filter. - {`{http_method=~".+"} | http_method = "GET"`, 21}, - {`{http_method=~".+"} | http_method = "HEAD", http_status = "500"`, 2}, + {`{http.method=~".+"} | http.method = "GET"`, 21}, + {`{http.method=~".+"} | http.method = "HEAD", http.status = "500"`, 2}, // Number of lines per protocol. // // "HTTP/1.0" 55 // "HTTP/1.1" 38 // "HTTP/2.0" 30 // - {`{http_method=~".+"} | json | protocol = "HTTP/1.0"`, 55}, - {`{http_method=~".+"} | json | protocol =~ "HTTP/1.\\d"`, 55 + 38}, - {`{http_method=~".+"} | json | protocol != "HTTP/2.0"`, 55 + 38}, - {`{http_method=~".+"} | json | protocol !~ "HTTP/2.\\d"`, 55 + 38}, + {`{http.method=~".+"} | json | protocol = "HTTP/1.0"`, 55}, + {`{http.method=~".+"} | json | protocol =~ "HTTP/1.\\d"`, 55 + 38}, + {`{http.method=~".+"} | json | protocol != "HTTP/2.0"`, 55 + 38}, + {`{http.method=~".+"} | json | protocol !~ "HTTP/2.\\d"`, 55 + 38}, // IP label filter. - {`{http_method="HEAD"} | json | host = "236.7.233.166"`, 1}, - {`{http_method="HEAD"} | json | host == ip("236.7.233.166")`, 1}, - {`{http_method="HEAD"} | json | host == ip("236.7.233.0/24")`, 1}, - {`{http_method="HEAD"} | json | host == ip("236.7.233.0-236.7.233.255")`, 1}, + {`{http.method="HEAD"} | json | host = "236.7.233.166"`, 1}, + {`{http.method="HEAD"} | json | host == ip("236.7.233.166")`, 1}, + {`{http.method="HEAD"} | json | host == ip("236.7.233.0/24")`, 1}, + {`{http.method="HEAD"} | json | host == ip("236.7.233.0-236.7.233.255")`, 1}, // Distinct filter. - {`{http_method=~".+"} | distinct http_method`, 6}, - {`{http_method=~".+"} | json | distinct method`, 6}, - {`{http_method=~".+"} | json | distinct protocol`, 3}, + {`{http.method=~".+"} | distinct http.method`, 6}, + {`{http.method=~".+"} | json | distinct method`, 6}, + {`{http.method=~".+"} | json | distinct protocol`, 3}, // Sure empty queries. - {`{http_method="GET"} | json | http_method != "GET"`, 0}, - {`{http_method="HEAD"} | clearly_not_exist > 0`, 0}, + {`{http.method="GET"} | json | http.method != "GET"`, 0}, + {`{http.method="HEAD"} | clearly_not_exist > 0`, 0}, } labelSetHasAttrs := func(t assert.TestingT, set lokiapi.LabelSet, attrs pcommon.Map) { // Do not check length, since label set may contain some parsed labels. @@ -245,7 +245,7 @@ func runTest( }) t.Run("MetricQueries", func(t *testing.T) { resp, err := c.QueryRange(ctx, lokiapi.QueryRangeParams{ - Query: `sum by (http_method) ( count_over_time({http_method=~".+"} [30s]) )`, + Query: `sum by (http.method) ( count_over_time({http.method=~".+"} [30s]) )`, // Query all data in a one step. Start: lokiapi.NewOptLokiTime(asLokiTime(set.End)), End: lokiapi.NewOptLokiTime(asLokiTime(set.End + otelstorage.Timestamp(10*time.Second))), @@ -262,9 +262,9 @@ func runTest( methods := map[string]string{} for _, series := range matrix { labels := series.Metric.Value - assert.Contains(t, labels, "http_method") + assert.Contains(t, labels, "http.method") assert.Len(t, labels, 1) - method := labels["http_method"] + method := labels["http.method"] values := series.Values assert.Len(t, values, 1) diff --git a/internal/chstorage/columns.go b/internal/chstorage/columns.go index da990e92..af9c6248 100644 --- a/internal/chstorage/columns.go +++ b/internal/chstorage/columns.go @@ -38,3 +38,9 @@ func (c tableColumns) Result() proto.Results { } return cols } + +func (c tableColumns) Reset() { + for _, col := range c { + col.Data.Reset() + } +} diff --git a/internal/chstorage/columns_logs.go b/internal/chstorage/columns_logs.go index 12aad13f..2fe32fea 100644 --- a/internal/chstorage/columns_logs.go +++ b/internal/chstorage/columns_logs.go @@ -168,9 +168,42 @@ func (c *logColumns) columns() tableColumns { func (c *logColumns) Input() proto.Input { return c.columns().Input() } func (c *logColumns) Result() proto.Results { return c.columns().Result() } +func (c *logColumns) Reset() { c.columns().Reset() } -func (c *logColumns) Reset() { - for _, col := range c.columns() { - col.Data.Reset() +type logAttrMapColumns struct { + name proto.ColStr // http_method + key proto.ColStr // http.method +} + +func newLogAttrMapColumns() *logAttrMapColumns { + return &logAttrMapColumns{} +} + +func (c *logAttrMapColumns) columns() tableColumns { + return []tableColumn{ + {Name: "name", Data: &c.name}, + {Name: "key", Data: &c.key}, } } + +func (c *logAttrMapColumns) Input() proto.Input { return c.columns().Input() } +func (c *logAttrMapColumns) Result() proto.Results { return c.columns().Result() } +func (c *logAttrMapColumns) Reset() { c.columns().Reset() } + +func (c *logAttrMapColumns) ForEach(f func(name, key string)) { + for i := 0; i < c.name.Rows(); i++ { + f(c.name.Row(i), c.key.Row(i)) + } +} + +func (c *logAttrMapColumns) AddAttrs(attrs otelstorage.Attrs) { + attrs.AsMap().Range(func(k string, v pcommon.Value) bool { + c.AddRow(otelstorage.KeyToLabel(k), k) + return true + }) +} + +func (c *logAttrMapColumns) AddRow(name, key string) { + c.name.Append(name) + c.key.Append(key) +} diff --git a/internal/chstorage/inserter_logs.go b/internal/chstorage/inserter_logs.go index 6bf049ed..9cf04047 100644 --- a/internal/chstorage/inserter_logs.go +++ b/internal/chstorage/inserter_logs.go @@ -9,6 +9,7 @@ import ( "go.opentelemetry.io/otel/trace" "github.com/go-faster/oteldb/internal/logstorage" + "github.com/go-faster/oteldb/internal/otelstorage" ) func (i *Inserter) mapRecords(c *logColumns, records []logstorage.Record) { @@ -18,9 +19,31 @@ func (i *Inserter) mapRecords(c *logColumns, records []logstorage.Record) { } // InsertLogLabels inserts given set of labels to the storage. -func (i *Inserter) InsertLogLabels(context.Context, map[logstorage.Label]struct{}) error { - // No-op. - // TODO(ernado): do we really need this or can just use materialized view? +func (i *Inserter) InsertLogLabels(ctx context.Context, set map[logstorage.Label]struct{}) (rerr error) { + table := i.tables.LogAttrs + ctx, span := i.tracer.Start(ctx, "InsertLogLabels", trace.WithAttributes( + attribute.Int("chstorage.labels_count", len(set)), + attribute.String("chstorage.table", table), + )) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + attrs := newLogAttrMapColumns() + for label := range set { + name := otelstorage.KeyToLabel(label.Name) + attrs.AddRow(name, label.Name) + } + if err := i.ch.Do(ctx, ch.Query{ + Body: attrs.Input().Into(table), + Input: attrs.Input(), + }); err != nil { + return errors.Wrap(err, "insert labels") + } + return nil } @@ -50,5 +73,18 @@ func (i *Inserter) InsertRecords(ctx context.Context, records []logstorage.Recor return errors.Wrap(err, "insert records") } + attrs := newLogAttrMapColumns() + for _, record := range records { + attrs.AddAttrs(record.Attrs) + attrs.AddAttrs(record.ResourceAttrs) + attrs.AddAttrs(record.ScopeAttrs) + } + if err := i.ch.Do(ctx, ch.Query{ + Body: attrs.Input().Into(i.tables.LogAttrs), + Input: attrs.Input(), + }); err != nil { + return errors.Wrap(err, "insert labels") + } + return nil } diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index cbd1ef8e..b7755fc7 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -108,6 +108,48 @@ func (l *labelStaticIterator) Next(t *logstorage.Label) bool { func (l *labelStaticIterator) Err() error { return nil } func (l *labelStaticIterator) Close() error { return nil } +func (q *Querier) getLabelMapping(ctx context.Context, labels []string) (_ map[string]string, rerr error) { + ctx, span := q.tracer.Start(ctx, "getLabelMapping", + trace.WithAttributes( + attribute.Int("chstorage.labels_count", len(labels)), + ), + ) + defer func() { + if rerr != nil { + span.RecordError(rerr) + } + span.End() + }() + + out := make(map[string]string, len(labels)) + attrs := newLogAttrMapColumns() + var inputData proto.ColStr + for _, label := range labels { + inputData.Append(label) + } + if err := q.ch.Do(ctx, ch.Query{ + Logger: zctx.From(ctx).Named("ch"), + Result: attrs.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + attrs.ForEach(func(name, key string) { + out[name] = key + }) + return nil + }, + ExternalTable: "_labels", + ExternalData: []proto.InputColumn{ + {Name: "name", Data: &inputData}, + }, + Body: fmt.Sprintf(`SELECT name, key FROM %[1]s INNER JOIN _labels ON (_labels.name = %[1]s.name)`, q.tables.LogAttrs), + }); err != nil { + return nil, errors.Wrap(err, "select") + } + + fmt.Println("mapped", labels, "to", out) + + return out, nil +} + // LabelValues implements logstorage.Querier. func (q *Querier) LabelValues(ctx context.Context, labelName string, opts logstorage.LabelsOptions) (_ iterators.Iterator[logstorage.Label], rerr error) { table := q.tables.Logs @@ -125,6 +167,15 @@ func (q *Querier) LabelValues(ctx context.Context, labelName string, opts logsto } span.End() }() + { + mapping, err := q.getLabelMapping(ctx, []string{labelName}) + if err != nil { + return nil, errors.Wrap(err, "get label mapping") + } + if key, ok := mapping[labelName]; ok { + labelName = key + } + } var ( names proto.ColStr out []jx.Raw @@ -201,6 +252,17 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta span.End() }() + // Gather all labels for mapping fetch. + var labels []string + for _, m := range params.Labels { + labels = append(labels, string(m.Label)) + } + mapping, err := q.getLabelMapping(ctx, labels) + if err != nil { + return nil, errors.Wrap(err, "get label mapping") + } + fmt.Println("mapping:", mapping) + out := newLogColumns() var query strings.Builder query.WriteString("SELECT ") @@ -212,6 +274,13 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta } fmt.Fprintf(&query, " FROM %s WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d)", table, start, end) for _, m := range params.Labels { + labelName := string(m.Label) + if key, ok := mapping[labelName]; ok { + fmt.Println("mapped", labelName, "to", key) + labelName = key + } else { + fmt.Println("no mapping for", labelName) + } switch m.Op { case logql.OpEq, logql.OpRe: query.WriteString(" AND (") @@ -231,9 +300,9 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta // TODO: how to match integers, booleans, floats, arrays? switch m.Op { case logql.OpEq, logql.OpNotEq: - fmt.Fprintf(&query, "JSONExtractString(%s, %s) = %s", column, singleQuoted(m.Label), singleQuoted(m.Value)) + fmt.Fprintf(&query, "JSONExtractString(%s, %s) = %s", column, singleQuoted(labelName), singleQuoted(m.Value)) case logql.OpRe, logql.OpNotRe: - fmt.Fprintf(&query, "JSONExtractString(%s, %s) REGEXP %s", column, singleQuoted(m.Label), singleQuoted(m.Value)) + fmt.Fprintf(&query, "JSONExtractString(%s, %s) REGEXP %s", column, singleQuoted(labelName), singleQuoted(m.Value)) } } query.WriteByte(')') @@ -258,7 +327,6 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta // TODO: use streaming. var data []logstorage.Record - if err := q.ch.Do(ctx, ch.Query{ Logger: zctx.From(ctx).Named("ch"), Body: query.String(), diff --git a/internal/chstorage/schema.go b/internal/chstorage/schema.go index 23760c8a..cce394c1 100644 --- a/internal/chstorage/schema.go +++ b/internal/chstorage/schema.go @@ -19,7 +19,8 @@ type Tables struct { Summaries string Labels string - Logs string + Logs string + LogAttrs string } // Validate checks table names @@ -48,6 +49,7 @@ func (t *Tables) Each(cb func(name *string) error) error { {&t.Labels, "Labels"}, {&t.Logs, "Logs"}, + {&t.LogAttrs, "LogAttrs"}, } { if err := cb(table.field); err != nil { return errors.Wrapf(err, "table %s", table.fieldName) @@ -68,7 +70,8 @@ func DefaultTables() Tables { Summaries: "metrics_summaries", Labels: "metrics_labels", - Logs: "logs", + Logs: "logs", + LogAttrs: "logs_attrs", } } @@ -97,6 +100,7 @@ func (t Tables) Create(ctx context.Context, c chClient) error { {t.Labels, labelsSchema}, {t.Logs, logsSchema}, + {t.LogAttrs, logAttrsSchema}, } { if err := c.Do(ctx, ch.Query{ Body: fmt.Sprintf(s.query, s.name), diff --git a/internal/chstorage/schema_logs.go b/internal/chstorage/schema_logs.go index ab76a90b..48fb2e85 100644 --- a/internal/chstorage/schema_logs.go +++ b/internal/chstorage/schema_logs.go @@ -34,4 +34,13 @@ CREATE TABLE IF NOT EXISTS %s ) ENGINE = MergeTree() ORDER BY (service_namespace, service_name, service_instance_id, toStartOfFiveMinutes(timestamp));` + + logAttrsSchema = ` +CREATE TABLE IF NOT EXISTS %s +( + name String, -- foo_bar + key String, -- foo.bar +) + ENGINE = ReplacingMergeTree + ORDER BY name;` ) diff --git a/internal/otelstorage/attrs.go b/internal/otelstorage/attrs.go index cdcaf759..8321fc6b 100644 --- a/internal/otelstorage/attrs.go +++ b/internal/otelstorage/attrs.go @@ -1,12 +1,19 @@ package otelstorage import ( + "strings" + "go.opentelemetry.io/collector/pdata/pcommon" "go.ytsaurus.tech/yt/go/yson" "github.com/go-faster/errors" ) +// KeyToLabel converts key to label name. +func KeyToLabel(key string) string { + return strings.ReplaceAll(key, ".", "_") +} + // Attrs is a YSON wrapper for attributes. type Attrs pcommon.Map From 00d21140e1ac142e04da4465cf48b54bcd5765c9 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Tue, 28 Nov 2023 15:38:01 +0300 Subject: [PATCH 072/112] chore: rm print --- internal/chstorage/querier_logs.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index b7755fc7..98823fe2 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -145,8 +145,6 @@ func (q *Querier) getLabelMapping(ctx context.Context, labels []string) (_ map[s return nil, errors.Wrap(err, "select") } - fmt.Println("mapped", labels, "to", out) - return out, nil } From a644670e834d3c351684e667139dd0821477110c Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Tue, 28 Nov 2023 15:42:37 +0300 Subject: [PATCH 073/112] test(lokie2e): undot --- integration/lokie2e/loki_e2e.go | 3 ++- integration/lokie2e/yt_test.go | 2 ++ internal/chstorage/querier_logs.go | 5 ++++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/integration/lokie2e/loki_e2e.go b/integration/lokie2e/loki_e2e.go index 3026382f..0dda10e9 100644 --- a/integration/lokie2e/loki_e2e.go +++ b/integration/lokie2e/loki_e2e.go @@ -114,5 +114,6 @@ func (s *BatchSet) addLabel(label logstorage.Label) { if s.Labels == nil { s.Labels = map[string][]logstorage.Label{} } - s.Labels[label.Name] = append(s.Labels[label.Name], label) + name := otelstorage.KeyToLabel(label.Name) + s.Labels[name] = append(s.Labels[name], label) } diff --git a/integration/lokie2e/yt_test.go b/integration/lokie2e/yt_test.go index 6a0c65ac..68caa866 100644 --- a/integration/lokie2e/yt_test.go +++ b/integration/lokie2e/yt_test.go @@ -23,6 +23,7 @@ import ( ) func TestYT(t *testing.T) { + t.Skip("Defer YT") t.Parallel() integration.Skip(t) ctx := context.Background() @@ -81,6 +82,7 @@ func TestYT(t *testing.T) { } func TestYTYQL(t *testing.T) { + t.Skip("Defer YT") t.Parallel() if os.Getenv("E2E") == "" { t.Skip("Set E2E env to run") diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index 98823fe2..ff412767 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -52,7 +52,10 @@ func (q *Querier) LabelNames(ctx context.Context, opts logstorage.LabelsOptions) }, OnResult: func(ctx context.Context, block proto.Block) error { for i := 0; i < names.Rows(); i++ { - out = append(out, names.Row(i)) + name := names.Row(i) + // TODO: add configuration option + name = otelstorage.KeyToLabel(name) + out = append(out, name) } return nil }, From 5749ea59354dee10f6718e04121e7885f4bdd6fd Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Tue, 28 Nov 2023 17:35:40 +0300 Subject: [PATCH 074/112] feat(oteldb.dev.local): add loki push --- dev/local/ch/docker-compose.yml | 5 +++++ dev/local/grafana/datasources.yaml | 23 ++++++++++++++++------- dev/local/otelcol.yml | 7 ++++++- 3 files changed, 27 insertions(+), 8 deletions(-) diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index d5c23410..5f2d873d 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -118,6 +118,11 @@ services: - ../prometheus:/etc/prometheus - prom:/prometheus + # for LogQL + loki: + image: grafana/loki:2.9.0 + command: -config.file=/etc/loki/local-config.yaml + # https://opentelemetry.io/docs/collector/installation/#docker-compose otelcol: image: ghcr.io/open-telemetry/opentelemetry-collector-releases/opentelemetry-collector-contrib:0.89.0 diff --git a/dev/local/grafana/datasources.yaml b/dev/local/grafana/datasources.yaml index cc8c295d..ad832e7b 100644 --- a/dev/local/grafana/datasources.yaml +++ b/dev/local/grafana/datasources.yaml @@ -1,7 +1,7 @@ apiVersion: 1 datasources: - - name: Tempo + - name: "oteldb: TraceQL" type: tempo access: proxy orgId: 1 @@ -12,18 +12,27 @@ datasources: serviceMap: datasourceUid: promoteldb - - name: Loki + - name: "oteldb: LogQL" type: loki access: proxy orgId: 1 url: http://oteldb:3100 + uid: loki-oteldb + jsonData: + serviceMap: + datasourceUid: prometheus + + - name: Loki + type: loki + access: proxy + orgId: 1 + url: http://loki:3100 uid: loki jsonData: - httpMethod: GET serviceMap: datasourceUid: promoteldb - - name: PromQL oteldb + - name: "oteldb: PromQL" type: prometheus access: proxy orgId: 1 @@ -52,7 +61,7 @@ datasources: access: proxy # For qryn - - name: Loki qryn + - name: "qryn: LogQL" type: loki access: proxy uid: loki-qryn @@ -69,7 +78,7 @@ datasources: name: traceID url: '$${__value.raw}' - - name: Tempo qryn + - name: "qryn: TraceQL" type: tempo access: proxy uid: tempo-qryn @@ -92,7 +101,7 @@ datasources: lokiSearch: datasourceUid: loki-qryn - - name: PromQL qryn + - name: "qryn: PromQL" type: prometheus access: proxy uid: prom-qryn diff --git a/dev/local/otelcol.yml b/dev/local/otelcol.yml index a4eb2542..3b0cd3a7 100644 --- a/dev/local/otelcol.yml +++ b/dev/local/otelcol.yml @@ -71,6 +71,11 @@ exporters: enabled: true # Convert resource attributes to metric labels tls: insecure: true + loki: + endpoint: http://loki:3100/loki/api/v1/push + default_labels_enabled: + exporter: false + job: true debug: sampling_initial: 5 sampling_thereafter: 200 @@ -94,7 +99,7 @@ service: logs: receivers: [otlp] processors: [batch] - exporters: [otlp, debug, otlp/qryn] + exporters: [otlp, otlp/qryn, loki] # https://github.com/open-telemetry/opentelemetry-collector/blob/main/docs/observability.md#how-we-expose-telemetry telemetry: metrics: From 69f4dbb591abbfaeaf9406606b364c3d5e559c3f Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Tue, 28 Nov 2023 17:50:01 +0300 Subject: [PATCH 075/112] fix(oteldb.dev.local): httpMethod: GET --- dev/local/grafana/datasources.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dev/local/grafana/datasources.yaml b/dev/local/grafana/datasources.yaml index ad832e7b..7dd69903 100644 --- a/dev/local/grafana/datasources.yaml +++ b/dev/local/grafana/datasources.yaml @@ -19,6 +19,7 @@ datasources: url: http://oteldb:3100 uid: loki-oteldb jsonData: + httpMethod: GET serviceMap: datasourceUid: prometheus @@ -29,6 +30,7 @@ datasources: url: http://loki:3100 uid: loki jsonData: + httpMethod: GET serviceMap: datasourceUid: promoteldb From aec8d9f0f3aa3a972bc0c93d6af1683149c85ad0 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Tue, 28 Nov 2023 17:53:57 +0300 Subject: [PATCH 076/112] feat(lokiapi): add indexStats endpoint --- _oas/loki.yml | 57 ++- internal/lokiapi/oas_client_gen.go | 204 ++++++--- internal/lokiapi/oas_faker_gen.go | 24 ++ internal/lokiapi/oas_handlers_gen.go | 141 ++++++- internal/lokiapi/oas_json_gen.go | 147 +++++++ internal/lokiapi/oas_parameters_gen.go | 386 ++++++++---------- internal/lokiapi/oas_response_decoders_gen.go | 83 ++++ internal/lokiapi/oas_response_encoders_gen.go | 14 + internal/lokiapi/oas_router_gen.go | 40 ++ internal/lokiapi/oas_schemas_gen.go | 48 +++ internal/lokiapi/oas_server_gen.go | 6 + .../lokiapi/oas_test_examples_gen_test.go | 12 + internal/lokiapi/oas_unimplemented_gen.go | 9 + internal/lokihandler/lokihandler.go | 9 + internal/lokiproxy/lokiproxy.go | 9 + 15 files changed, 882 insertions(+), 307 deletions(-) diff --git a/_oas/loki.yml b/_oas/loki.yml index 2f74be71..2608eb1e 100644 --- a/_oas/loki.yml +++ b/_oas/loki.yml @@ -13,7 +13,6 @@ paths: operationId: query description: Query parameters: - - $ref: "#/components/parameters/GrafanaUser" - name: query description: The LogQL query to perform. in: query @@ -59,7 +58,6 @@ paths: operationId: queryRange description: Query range parameters: - - $ref: "#/components/parameters/GrafanaUser" - name: start in: query schema: @@ -126,7 +124,6 @@ paths: Get labels. Used by Grafana to test connection to Loki. parameters: - - $ref: "#/components/parameters/GrafanaUser" - name: start description: | The start time for the query as a nanosecond Unix epoch. @@ -163,7 +160,6 @@ paths: operationId: labelValues description: Get values of label parameters: - - $ref: "#/components/parameters/GrafanaUser" - name: start description: | The start time for the query as a nanosecond Unix epoch. @@ -262,16 +258,35 @@ paths: description: Successful push default: $ref: "#/components/responses/Error" + /loki/api/v1/index/stats: + get: + operationId: indexStats + description: Get index stats + parameters: + - name: start + in: query + schema: + $ref: "#/components/schemas/LokiTime" + - name: end + in: query + schema: + $ref: "#/components/schemas/LokiTime" + - name: query + description: The LogQL matchers to check. + in: query + required: true + schema: + type: string + responses: + "200": + description: Index stats + content: + application/json: + schema: + $ref: "#/components/schemas/IndexStats" + default: + $ref: "#/components/responses/Error" components: - parameters: - # TODO(ernado): extract to separate file? Same for all datasources - GrafanaUser: - name: X-Grafana-User - in: header - required: false - description: "Grafana username that is passed to datasource when making requests from Grafana. Used for authentication and authorization." - schema: - type: string responses: Error: description: Error while processing request @@ -402,6 +417,22 @@ components: - type: string description: value x-ogen-name: V + IndexStats: + type: object + required: + - streams + - chunks + - entries + - bytes + properties: + streams: + type: integer + chunks: + type: integer + entries: + type: integer + bytes: + type: integer Stats: type: object Direction: diff --git a/internal/lokiapi/oas_client_gen.go b/internal/lokiapi/oas_client_gen.go index 3d20cc18..ef7bcb39 100644 --- a/internal/lokiapi/oas_client_gen.go +++ b/internal/lokiapi/oas_client_gen.go @@ -23,6 +23,12 @@ import ( // Invoker invokes operations described by OpenAPI v3 specification. type Invoker interface { + // IndexStats invokes indexStats operation. + // + // Get index stats. + // + // GET /loki/api/v1/index/stats + IndexStats(ctx context.Context, params IndexStatsParams) (*IndexStats, error) // LabelValues invokes labelValues operation. // // Get values of label. @@ -114,6 +120,136 @@ func (c *Client) requestURL(ctx context.Context) *url.URL { return u } +// IndexStats invokes indexStats operation. +// +// Get index stats. +// +// GET /loki/api/v1/index/stats +func (c *Client) IndexStats(ctx context.Context, params IndexStatsParams) (*IndexStats, error) { + res, err := c.sendIndexStats(ctx, params) + return res, err +} + +func (c *Client) sendIndexStats(ctx context.Context, params IndexStatsParams) (res *IndexStats, err error) { + otelAttrs := []attribute.KeyValue{ + otelogen.OperationID("indexStats"), + semconv.HTTPMethodKey.String("GET"), + semconv.HTTPRouteKey.String("/loki/api/v1/index/stats"), + } + + // Run stopwatch. + startTime := time.Now() + defer func() { + // Use floating point division here for higher precision (instead of Millisecond method). + elapsedDuration := time.Since(startTime) + c.duration.Record(ctx, float64(float64(elapsedDuration)/float64(time.Millisecond)), metric.WithAttributes(otelAttrs...)) + }() + + // Increment request counter. + c.requests.Add(ctx, 1, metric.WithAttributes(otelAttrs...)) + + // Start a span for this request. + ctx, span := c.cfg.Tracer.Start(ctx, "IndexStats", + trace.WithAttributes(otelAttrs...), + clientSpanKind, + ) + // Track stage for error reporting. + var stage string + defer func() { + if err != nil { + span.RecordError(err) + span.SetStatus(codes.Error, stage) + c.errors.Add(ctx, 1, metric.WithAttributes(otelAttrs...)) + } + span.End() + }() + + stage = "BuildURL" + u := uri.Clone(c.requestURL(ctx)) + var pathParts [1]string + pathParts[0] = "/loki/api/v1/index/stats" + uri.AddPathParts(u, pathParts[:]...) + + stage = "EncodeQueryParams" + q := uri.NewQueryEncoder() + { + // Encode "start" parameter. + cfg := uri.QueryParameterEncodingConfig{ + Name: "start", + Style: uri.QueryStyleForm, + Explode: true, + } + + if err := q.EncodeParam(cfg, func(e uri.Encoder) error { + if val, ok := params.Start.Get(); ok { + if unwrapped := string(val); true { + return e.EncodeValue(conv.StringToString(unwrapped)) + } + return nil + } + return nil + }); err != nil { + return res, errors.Wrap(err, "encode query") + } + } + { + // Encode "end" parameter. + cfg := uri.QueryParameterEncodingConfig{ + Name: "end", + Style: uri.QueryStyleForm, + Explode: true, + } + + if err := q.EncodeParam(cfg, func(e uri.Encoder) error { + if val, ok := params.End.Get(); ok { + if unwrapped := string(val); true { + return e.EncodeValue(conv.StringToString(unwrapped)) + } + return nil + } + return nil + }); err != nil { + return res, errors.Wrap(err, "encode query") + } + } + { + // Encode "query" parameter. + cfg := uri.QueryParameterEncodingConfig{ + Name: "query", + Style: uri.QueryStyleForm, + Explode: true, + } + + if err := q.EncodeParam(cfg, func(e uri.Encoder) error { + return e.EncodeValue(conv.StringToString(params.Query)) + }); err != nil { + return res, errors.Wrap(err, "encode query") + } + } + u.RawQuery = q.Values().Encode() + + stage = "EncodeRequest" + r, err := ht.NewRequest(ctx, "GET", u) + if err != nil { + return res, errors.Wrap(err, "create request") + } + + stage = "SendRequest" + resp, err := c.cfg.Client.Do(r) + if err != nil { + return res, errors.Wrap(err, "do request") + } + defer resp.Body.Close() + + stage = "DecodeResponse" + result, err := decodeIndexStatsResponse(resp) + if err != nil { + return res, errors.Wrap(err, "decode response") + } + + return result, nil +} + // LabelValues invokes labelValues operation. // // Get values of label. @@ -270,23 +406,6 @@ func (c *Client) sendLabelValues(ctx context.Context, params LabelValuesParams) return res, errors.Wrap(err, "create request") } - stage = "EncodeHeaderParams" - h := uri.NewHeaderEncoder(r.Header) - { - cfg := uri.HeaderParameterEncodingConfig{ - Name: "X-Grafana-User", - Explode: false, - } - if err := h.EncodeParam(cfg, func(e uri.Encoder) error { - if val, ok := params.XGrafanaUser.Get(); ok { - return e.EncodeValue(conv.StringToString(val)) - } - return nil - }); err != nil { - return res, errors.Wrap(err, "encode header") - } - } - stage = "SendRequest" resp, err := c.cfg.Client.Do(r) if err != nil { @@ -424,23 +543,6 @@ func (c *Client) sendLabels(ctx context.Context, params LabelsParams) (res *Labe return res, errors.Wrap(err, "create request") } - stage = "EncodeHeaderParams" - h := uri.NewHeaderEncoder(r.Header) - { - cfg := uri.HeaderParameterEncodingConfig{ - Name: "X-Grafana-User", - Explode: false, - } - if err := h.EncodeParam(cfg, func(e uri.Encoder) error { - if val, ok := params.XGrafanaUser.Get(); ok { - return e.EncodeValue(conv.StringToString(val)) - } - return nil - }); err != nil { - return res, errors.Wrap(err, "encode header") - } - } - stage = "SendRequest" resp, err := c.cfg.Client.Do(r) if err != nil { @@ -660,23 +762,6 @@ func (c *Client) sendQuery(ctx context.Context, params QueryParams) (res *QueryR return res, errors.Wrap(err, "create request") } - stage = "EncodeHeaderParams" - h := uri.NewHeaderEncoder(r.Header) - { - cfg := uri.HeaderParameterEncodingConfig{ - Name: "X-Grafana-User", - Explode: false, - } - if err := h.EncodeParam(cfg, func(e uri.Encoder) error { - if val, ok := params.XGrafanaUser.Get(); ok { - return e.EncodeValue(conv.StringToString(val)) - } - return nil - }); err != nil { - return res, errors.Wrap(err, "encode header") - } - } - stage = "SendRequest" resp, err := c.cfg.Client.Do(r) if err != nil { @@ -881,23 +966,6 @@ func (c *Client) sendQueryRange(ctx context.Context, params QueryRangeParams) (r return res, errors.Wrap(err, "create request") } - stage = "EncodeHeaderParams" - h := uri.NewHeaderEncoder(r.Header) - { - cfg := uri.HeaderParameterEncodingConfig{ - Name: "X-Grafana-User", - Explode: false, - } - if err := h.EncodeParam(cfg, func(e uri.Encoder) error { - if val, ok := params.XGrafanaUser.Get(); ok { - return e.EncodeValue(conv.StringToString(val)) - } - return nil - }); err != nil { - return res, errors.Wrap(err, "encode header") - } - } - stage = "SendRequest" resp, err := c.cfg.Client.Do(r) if err != nil { diff --git a/internal/lokiapi/oas_faker_gen.go b/internal/lokiapi/oas_faker_gen.go index 0f34e331..f794bb8f 100644 --- a/internal/lokiapi/oas_faker_gen.go +++ b/internal/lokiapi/oas_faker_gen.go @@ -29,6 +29,30 @@ func (s *FPoint) SetFake() { } } +// SetFake set fake values. +func (s *IndexStats) SetFake() { + { + { + s.Streams = int(0) + } + } + { + { + s.Chunks = int(0) + } + } + { + { + s.Entries = int(0) + } + } + { + { + s.Bytes = int(0) + } + } +} + // SetFake set fake values. func (s *LabelSet) SetFake() { var ( diff --git a/internal/lokiapi/oas_handlers_gen.go b/internal/lokiapi/oas_handlers_gen.go index 5b57af5e..e2e790f4 100644 --- a/internal/lokiapi/oas_handlers_gen.go +++ b/internal/lokiapi/oas_handlers_gen.go @@ -20,6 +20,131 @@ import ( "github.com/ogen-go/ogen/otelogen" ) +// handleIndexStatsRequest handles indexStats operation. +// +// Get index stats. +// +// GET /loki/api/v1/index/stats +func (s *Server) handleIndexStatsRequest(args [0]string, argsEscaped bool, w http.ResponseWriter, r *http.Request) { + otelAttrs := []attribute.KeyValue{ + otelogen.OperationID("indexStats"), + semconv.HTTPMethodKey.String("GET"), + semconv.HTTPRouteKey.String("/loki/api/v1/index/stats"), + } + + // Start a span for this request. + ctx, span := s.cfg.Tracer.Start(r.Context(), "IndexStats", + trace.WithAttributes(otelAttrs...), + serverSpanKind, + ) + defer span.End() + + // Run stopwatch. + startTime := time.Now() + defer func() { + elapsedDuration := time.Since(startTime) + // Use floating point division here for higher precision (instead of Millisecond method). + s.duration.Record(ctx, float64(float64(elapsedDuration)/float64(time.Millisecond)), metric.WithAttributes(otelAttrs...)) + }() + + // Increment request counter. + s.requests.Add(ctx, 1, metric.WithAttributes(otelAttrs...)) + + var ( + recordError = func(stage string, err error) { + span.RecordError(err) + span.SetStatus(codes.Error, stage) + s.errors.Add(ctx, 1, metric.WithAttributes(otelAttrs...)) + } + err error + opErrContext = ogenerrors.OperationContext{ + Name: "IndexStats", + ID: "indexStats", + } + ) + params, err := decodeIndexStatsParams(args, argsEscaped, r) + if err != nil { + err = &ogenerrors.DecodeParamsError{ + OperationContext: opErrContext, + Err: err, + } + recordError("DecodeParams", err) + s.cfg.ErrorHandler(ctx, w, r, err) + return + } + + var response *IndexStats + if m := s.cfg.Middleware; m != nil { + mreq := middleware.Request{ + Context: ctx, + OperationName: "IndexStats", + OperationSummary: "", + OperationID: "indexStats", + Body: nil, + Params: middleware.Parameters{ + { + Name: "start", + In: "query", + }: params.Start, + { + Name: "end", + In: "query", + }: params.End, + { + Name: "query", + In: "query", + }: params.Query, + }, + Raw: r, + } + + type ( + Request = struct{} + Params = IndexStatsParams + Response = *IndexStats + ) + response, err = middleware.HookMiddleware[ + Request, + Params, + Response, + ]( + m, + mreq, + unpackIndexStatsParams, + func(ctx context.Context, request Request, params Params) (response Response, err error) { + response, err = s.h.IndexStats(ctx, params) + return response, err + }, + ) + } else { + response, err = s.h.IndexStats(ctx, params) + } + if err != nil { + if errRes, ok := errors.Into[*ErrorStatusCode](err); ok { + if err := encodeErrorResponse(errRes, w, span); err != nil { + recordError("Internal", err) + } + return + } + if errors.Is(err, ht.ErrNotImplemented) { + s.cfg.ErrorHandler(ctx, w, r, err) + return + } + if err := encodeErrorResponse(s.h.NewError(ctx, err), w, span); err != nil { + recordError("Internal", err) + } + return + } + + if err := encodeIndexStatsResponse(response, w, span); err != nil { + recordError("EncodeResponse", err) + if !errors.Is(err, ht.ErrInternalServerErrorResponse) { + s.cfg.ErrorHandler(ctx, w, r, err) + } + return + } +} + // handleLabelValuesRequest handles labelValues operation. // // Get values of label. @@ -82,10 +207,6 @@ func (s *Server) handleLabelValuesRequest(args [1]string, argsEscaped bool, w ht OperationID: "labelValues", Body: nil, Params: middleware.Parameters{ - { - Name: "X-Grafana-User", - In: "header", - }: params.XGrafanaUser, { Name: "start", In: "query", @@ -220,10 +341,6 @@ func (s *Server) handleLabelsRequest(args [0]string, argsEscaped bool, w http.Re OperationID: "labels", Body: nil, Params: middleware.Parameters{ - { - Name: "X-Grafana-User", - In: "header", - }: params.XGrafanaUser, { Name: "start", In: "query", @@ -466,10 +583,6 @@ func (s *Server) handleQueryRequest(args [0]string, argsEscaped bool, w http.Res OperationID: "query", Body: nil, Params: middleware.Parameters{ - { - Name: "X-Grafana-User", - In: "header", - }: params.XGrafanaUser, { Name: "query", In: "query", @@ -599,10 +712,6 @@ func (s *Server) handleQueryRangeRequest(args [0]string, argsEscaped bool, w htt OperationID: "queryRange", Body: nil, Params: middleware.Parameters{ - { - Name: "X-Grafana-User", - In: "header", - }: params.XGrafanaUser, { Name: "start", In: "query", diff --git a/internal/lokiapi/oas_json_gen.go b/internal/lokiapi/oas_json_gen.go index a3da7ab8..ee185856 100644 --- a/internal/lokiapi/oas_json_gen.go +++ b/internal/lokiapi/oas_json_gen.go @@ -121,6 +121,153 @@ func (s *FPoint) UnmarshalJSON(data []byte) error { return s.Decode(d) } +// Encode implements json.Marshaler. +func (s *IndexStats) Encode(e *jx.Encoder) { + e.ObjStart() + s.encodeFields(e) + e.ObjEnd() +} + +// encodeFields encodes fields. +func (s *IndexStats) encodeFields(e *jx.Encoder) { + { + e.FieldStart("streams") + e.Int(s.Streams) + } + { + e.FieldStart("chunks") + e.Int(s.Chunks) + } + { + e.FieldStart("entries") + e.Int(s.Entries) + } + { + e.FieldStart("bytes") + e.Int(s.Bytes) + } +} + +var jsonFieldsNameOfIndexStats = [4]string{ + 0: "streams", + 1: "chunks", + 2: "entries", + 3: "bytes", +} + +// Decode decodes IndexStats from json. +func (s *IndexStats) Decode(d *jx.Decoder) error { + if s == nil { + return errors.New("invalid: unable to decode IndexStats to nil") + } + var requiredBitSet [1]uint8 + + if err := d.ObjBytes(func(d *jx.Decoder, k []byte) error { + switch string(k) { + case "streams": + requiredBitSet[0] |= 1 << 0 + if err := func() error { + v, err := d.Int() + s.Streams = int(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"streams\"") + } + case "chunks": + requiredBitSet[0] |= 1 << 1 + if err := func() error { + v, err := d.Int() + s.Chunks = int(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"chunks\"") + } + case "entries": + requiredBitSet[0] |= 1 << 2 + if err := func() error { + v, err := d.Int() + s.Entries = int(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"entries\"") + } + case "bytes": + requiredBitSet[0] |= 1 << 3 + if err := func() error { + v, err := d.Int() + s.Bytes = int(v) + if err != nil { + return err + } + return nil + }(); err != nil { + return errors.Wrap(err, "decode field \"bytes\"") + } + default: + return d.Skip() + } + return nil + }); err != nil { + return errors.Wrap(err, "decode IndexStats") + } + // Validate required fields. + var failures []validate.FieldError + for i, mask := range [1]uint8{ + 0b00001111, + } { + if result := (requiredBitSet[i] & mask) ^ mask; result != 0 { + // Mask only required fields and check equality to mask using XOR. + // + // If XOR result is not zero, result is not equal to expected, so some fields are missed. + // Bits of fields which would be set are actually bits of missed fields. + missed := bits.OnesCount8(result) + for bitN := 0; bitN < missed; bitN++ { + bitIdx := bits.TrailingZeros8(result) + fieldIdx := i*8 + bitIdx + var name string + if fieldIdx < len(jsonFieldsNameOfIndexStats) { + name = jsonFieldsNameOfIndexStats[fieldIdx] + } else { + name = strconv.Itoa(fieldIdx) + } + failures = append(failures, validate.FieldError{ + Name: name, + Error: validate.ErrFieldRequired, + }) + // Reset bit. + result &^= 1 << bitIdx + } + } + } + if len(failures) > 0 { + return &validate.Error{Fields: failures} + } + + return nil +} + +// MarshalJSON implements stdjson.Marshaler. +func (s *IndexStats) MarshalJSON() ([]byte, error) { + e := jx.Encoder{} + s.Encode(&e) + return e.Bytes(), nil +} + +// UnmarshalJSON implements stdjson.Unmarshaler. +func (s *IndexStats) UnmarshalJSON(data []byte) error { + d := jx.DecodeBytes(data) + return s.Decode(d) +} + // Encode implements json.Marshaler. func (s LabelSet) Encode(e *jx.Encoder) { e.ObjStart() diff --git a/internal/lokiapi/oas_parameters_gen.go b/internal/lokiapi/oas_parameters_gen.go index 198933d3..394d30e4 100644 --- a/internal/lokiapi/oas_parameters_gen.go +++ b/internal/lokiapi/oas_parameters_gen.go @@ -15,11 +15,182 @@ import ( "github.com/ogen-go/ogen/validate" ) +// IndexStatsParams is parameters of indexStats operation. +type IndexStatsParams struct { + Start OptLokiTime + End OptLokiTime + // The LogQL matchers to check. + Query string +} + +func unpackIndexStatsParams(packed middleware.Parameters) (params IndexStatsParams) { + { + key := middleware.ParameterKey{ + Name: "start", + In: "query", + } + if v, ok := packed[key]; ok { + params.Start = v.(OptLokiTime) + } + } + { + key := middleware.ParameterKey{ + Name: "end", + In: "query", + } + if v, ok := packed[key]; ok { + params.End = v.(OptLokiTime) + } + } + { + key := middleware.ParameterKey{ + Name: "query", + In: "query", + } + params.Query = packed[key].(string) + } + return params +} + +func decodeIndexStatsParams(args [0]string, argsEscaped bool, r *http.Request) (params IndexStatsParams, _ error) { + q := uri.NewQueryDecoder(r.URL.Query()) + // Decode query: start. + if err := func() error { + cfg := uri.QueryParameterDecodingConfig{ + Name: "start", + Style: uri.QueryStyleForm, + Explode: true, + } + + if err := q.HasParam(cfg); err == nil { + if err := q.DecodeParam(cfg, func(d uri.Decoder) error { + var paramsDotStartVal LokiTime + if err := func() error { + var paramsDotStartValVal string + if err := func() error { + val, err := d.DecodeValue() + if err != nil { + return err + } + + c, err := conv.ToString(val) + if err != nil { + return err + } + + paramsDotStartValVal = c + return nil + }(); err != nil { + return err + } + paramsDotStartVal = LokiTime(paramsDotStartValVal) + return nil + }(); err != nil { + return err + } + params.Start.SetTo(paramsDotStartVal) + return nil + }); err != nil { + return err + } + } + return nil + }(); err != nil { + return params, &ogenerrors.DecodeParamError{ + Name: "start", + In: "query", + Err: err, + } + } + // Decode query: end. + if err := func() error { + cfg := uri.QueryParameterDecodingConfig{ + Name: "end", + Style: uri.QueryStyleForm, + Explode: true, + } + + if err := q.HasParam(cfg); err == nil { + if err := q.DecodeParam(cfg, func(d uri.Decoder) error { + var paramsDotEndVal LokiTime + if err := func() error { + var paramsDotEndValVal string + if err := func() error { + val, err := d.DecodeValue() + if err != nil { + return err + } + + c, err := conv.ToString(val) + if err != nil { + return err + } + + paramsDotEndValVal = c + return nil + }(); err != nil { + return err + } + paramsDotEndVal = LokiTime(paramsDotEndValVal) + return nil + }(); err != nil { + return err + } + params.End.SetTo(paramsDotEndVal) + return nil + }); err != nil { + return err + } + } + return nil + }(); err != nil { + return params, &ogenerrors.DecodeParamError{ + Name: "end", + In: "query", + Err: err, + } + } + // Decode query: query. + if err := func() error { + cfg := uri.QueryParameterDecodingConfig{ + Name: "query", + Style: uri.QueryStyleForm, + Explode: true, + } + + if err := q.HasParam(cfg); err == nil { + if err := q.DecodeParam(cfg, func(d uri.Decoder) error { + val, err := d.DecodeValue() + if err != nil { + return err + } + + c, err := conv.ToString(val) + if err != nil { + return err + } + + params.Query = c + return nil + }); err != nil { + return err + } + } else { + return validate.ErrFieldRequired + } + return nil + }(); err != nil { + return params, &ogenerrors.DecodeParamError{ + Name: "query", + In: "query", + Err: err, + } + } + return params, nil +} + // LabelValuesParams is parameters of labelValues operation. type LabelValuesParams struct { - // Grafana username that is passed to datasource when making requests from Grafana. Used for - // authentication and authorization. - XGrafanaUser OptString // The start time for the query as a nanosecond Unix epoch. // Defaults to 6 hours ago. Start OptLokiTime @@ -38,15 +209,6 @@ type LabelValuesParams struct { } func unpackLabelValuesParams(packed middleware.Parameters) (params LabelValuesParams) { - { - key := middleware.ParameterKey{ - Name: "X-Grafana-User", - In: "header", - } - if v, ok := packed[key]; ok { - params.XGrafanaUser = v.(OptString) - } - } { key := middleware.ParameterKey{ Name: "start", @@ -95,46 +257,6 @@ func unpackLabelValuesParams(packed middleware.Parameters) (params LabelValuesPa func decodeLabelValuesParams(args [1]string, argsEscaped bool, r *http.Request) (params LabelValuesParams, _ error) { q := uri.NewQueryDecoder(r.URL.Query()) - h := uri.NewHeaderDecoder(r.Header) - // Decode header: X-Grafana-User. - if err := func() error { - cfg := uri.HeaderParameterDecodingConfig{ - Name: "X-Grafana-User", - Explode: false, - } - if err := h.HasParam(cfg); err == nil { - if err := h.DecodeParam(cfg, func(d uri.Decoder) error { - var paramsDotXGrafanaUserVal string - if err := func() error { - val, err := d.DecodeValue() - if err != nil { - return err - } - - c, err := conv.ToString(val) - if err != nil { - return err - } - - paramsDotXGrafanaUserVal = c - return nil - }(); err != nil { - return err - } - params.XGrafanaUser.SetTo(paramsDotXGrafanaUserVal) - return nil - }); err != nil { - return err - } - } - return nil - }(); err != nil { - return params, &ogenerrors.DecodeParamError{ - Name: "X-Grafana-User", - In: "header", - Err: err, - } - } // Decode query: start. if err := func() error { cfg := uri.QueryParameterDecodingConfig{ @@ -385,9 +507,6 @@ func decodeLabelValuesParams(args [1]string, argsEscaped bool, r *http.Request) // LabelsParams is parameters of labels operation. type LabelsParams struct { - // Grafana username that is passed to datasource when making requests from Grafana. Used for - // authentication and authorization. - XGrafanaUser OptString // The start time for the query as a nanosecond Unix epoch. // Defaults to 6 hours ago. Start OptLokiTime @@ -401,15 +520,6 @@ type LabelsParams struct { } func unpackLabelsParams(packed middleware.Parameters) (params LabelsParams) { - { - key := middleware.ParameterKey{ - Name: "X-Grafana-User", - In: "header", - } - if v, ok := packed[key]; ok { - params.XGrafanaUser = v.(OptString) - } - } { key := middleware.ParameterKey{ Name: "start", @@ -442,46 +552,6 @@ func unpackLabelsParams(packed middleware.Parameters) (params LabelsParams) { func decodeLabelsParams(args [0]string, argsEscaped bool, r *http.Request) (params LabelsParams, _ error) { q := uri.NewQueryDecoder(r.URL.Query()) - h := uri.NewHeaderDecoder(r.Header) - // Decode header: X-Grafana-User. - if err := func() error { - cfg := uri.HeaderParameterDecodingConfig{ - Name: "X-Grafana-User", - Explode: false, - } - if err := h.HasParam(cfg); err == nil { - if err := h.DecodeParam(cfg, func(d uri.Decoder) error { - var paramsDotXGrafanaUserVal string - if err := func() error { - val, err := d.DecodeValue() - if err != nil { - return err - } - - c, err := conv.ToString(val) - if err != nil { - return err - } - - paramsDotXGrafanaUserVal = c - return nil - }(); err != nil { - return err - } - params.XGrafanaUser.SetTo(paramsDotXGrafanaUserVal) - return nil - }); err != nil { - return err - } - } - return nil - }(); err != nil { - return params, &ogenerrors.DecodeParamError{ - Name: "X-Grafana-User", - In: "header", - Err: err, - } - } // Decode query: start. if err := func() error { cfg := uri.QueryParameterDecodingConfig{ @@ -646,9 +716,6 @@ func decodeLabelsParams(args [0]string, argsEscaped bool, r *http.Request) (para // QueryParams is parameters of query operation. type QueryParams struct { - // Grafana username that is passed to datasource when making requests from Grafana. Used for - // authentication and authorization. - XGrafanaUser OptString // The LogQL query to perform. Query string // The max number of entries to return. @@ -665,15 +732,6 @@ type QueryParams struct { } func unpackQueryParams(packed middleware.Parameters) (params QueryParams) { - { - key := middleware.ParameterKey{ - Name: "X-Grafana-User", - In: "header", - } - if v, ok := packed[key]; ok { - params.XGrafanaUser = v.(OptString) - } - } { key := middleware.ParameterKey{ Name: "query", @@ -713,46 +771,6 @@ func unpackQueryParams(packed middleware.Parameters) (params QueryParams) { func decodeQueryParams(args [0]string, argsEscaped bool, r *http.Request) (params QueryParams, _ error) { q := uri.NewQueryDecoder(r.URL.Query()) - h := uri.NewHeaderDecoder(r.Header) - // Decode header: X-Grafana-User. - if err := func() error { - cfg := uri.HeaderParameterDecodingConfig{ - Name: "X-Grafana-User", - Explode: false, - } - if err := h.HasParam(cfg); err == nil { - if err := h.DecodeParam(cfg, func(d uri.Decoder) error { - var paramsDotXGrafanaUserVal string - if err := func() error { - val, err := d.DecodeValue() - if err != nil { - return err - } - - c, err := conv.ToString(val) - if err != nil { - return err - } - - paramsDotXGrafanaUserVal = c - return nil - }(); err != nil { - return err - } - params.XGrafanaUser.SetTo(paramsDotXGrafanaUserVal) - return nil - }); err != nil { - return err - } - } - return nil - }(); err != nil { - return params, &ogenerrors.DecodeParamError{ - Name: "X-Grafana-User", - In: "header", - Err: err, - } - } // Decode query: query. if err := func() error { cfg := uri.QueryParameterDecodingConfig{ @@ -963,11 +981,8 @@ func decodeQueryParams(args [0]string, argsEscaped bool, r *http.Request) (param // QueryRangeParams is parameters of queryRange operation. type QueryRangeParams struct { - // Grafana username that is passed to datasource when making requests from Grafana. Used for - // authentication and authorization. - XGrafanaUser OptString - Start OptLokiTime - End OptLokiTime + Start OptLokiTime + End OptLokiTime // A `duration` used to calculate `start` relative to `end`. // If `end` is in the future, `start` is calculated as this duration before now. // Any value specified for start supersedes this parameter. @@ -991,15 +1006,6 @@ type QueryRangeParams struct { } func unpackQueryRangeParams(packed middleware.Parameters) (params QueryRangeParams) { - { - key := middleware.ParameterKey{ - Name: "X-Grafana-User", - In: "header", - } - if v, ok := packed[key]; ok { - params.XGrafanaUser = v.(OptString) - } - } { key := middleware.ParameterKey{ Name: "start", @@ -1066,46 +1072,6 @@ func unpackQueryRangeParams(packed middleware.Parameters) (params QueryRangePara func decodeQueryRangeParams(args [0]string, argsEscaped bool, r *http.Request) (params QueryRangeParams, _ error) { q := uri.NewQueryDecoder(r.URL.Query()) - h := uri.NewHeaderDecoder(r.Header) - // Decode header: X-Grafana-User. - if err := func() error { - cfg := uri.HeaderParameterDecodingConfig{ - Name: "X-Grafana-User", - Explode: false, - } - if err := h.HasParam(cfg); err == nil { - if err := h.DecodeParam(cfg, func(d uri.Decoder) error { - var paramsDotXGrafanaUserVal string - if err := func() error { - val, err := d.DecodeValue() - if err != nil { - return err - } - - c, err := conv.ToString(val) - if err != nil { - return err - } - - paramsDotXGrafanaUserVal = c - return nil - }(); err != nil { - return err - } - params.XGrafanaUser.SetTo(paramsDotXGrafanaUserVal) - return nil - }); err != nil { - return err - } - } - return nil - }(); err != nil { - return params, &ogenerrors.DecodeParamError{ - Name: "X-Grafana-User", - In: "header", - Err: err, - } - } // Decode query: start. if err := func() error { cfg := uri.QueryParameterDecodingConfig{ diff --git a/internal/lokiapi/oas_response_decoders_gen.go b/internal/lokiapi/oas_response_decoders_gen.go index 679e144d..ca5e964e 100644 --- a/internal/lokiapi/oas_response_decoders_gen.go +++ b/internal/lokiapi/oas_response_decoders_gen.go @@ -14,6 +14,89 @@ import ( "github.com/ogen-go/ogen/validate" ) +func decodeIndexStatsResponse(resp *http.Response) (res *IndexStats, _ error) { + switch resp.StatusCode { + case 200: + // Code 200. + ct, _, err := mime.ParseMediaType(resp.Header.Get("Content-Type")) + if err != nil { + return res, errors.Wrap(err, "parse media type") + } + switch { + case ct == "application/json": + buf, err := io.ReadAll(resp.Body) + if err != nil { + return res, err + } + d := jx.DecodeBytes(buf) + + var response IndexStats + if err := func() error { + if err := response.Decode(d); err != nil { + return err + } + if err := d.Skip(); err != io.EOF { + return errors.New("unexpected trailing data") + } + return nil + }(); err != nil { + err = &ogenerrors.DecodeBodyError{ + ContentType: ct, + Body: buf, + Err: err, + } + return res, err + } + return &response, nil + default: + return res, validate.InvalidContentType(ct) + } + } + // Convenient error response. + defRes, err := func() (res *ErrorStatusCode, err error) { + ct, _, err := mime.ParseMediaType(resp.Header.Get("Content-Type")) + if err != nil { + return res, errors.Wrap(err, "parse media type") + } + switch { + case ct == "application/json": + buf, err := io.ReadAll(resp.Body) + if err != nil { + return res, err + } + d := jx.DecodeBytes(buf) + + var response Error + if err := func() error { + if err := response.Decode(d); err != nil { + return err + } + if err := d.Skip(); err != io.EOF { + return errors.New("unexpected trailing data") + } + return nil + }(); err != nil { + err = &ogenerrors.DecodeBodyError{ + ContentType: ct, + Body: buf, + Err: err, + } + return res, err + } + return &ErrorStatusCode{ + StatusCode: resp.StatusCode, + Response: response, + }, nil + default: + return res, validate.InvalidContentType(ct) + } + }() + if err != nil { + return res, errors.Wrapf(err, "default (code %d)", resp.StatusCode) + } + return res, errors.Wrap(defRes, "error") +} + func decodeLabelValuesResponse(resp *http.Response) (res *Values, _ error) { switch resp.StatusCode { case 200: diff --git a/internal/lokiapi/oas_response_encoders_gen.go b/internal/lokiapi/oas_response_encoders_gen.go index 52007b60..d49f5a8b 100644 --- a/internal/lokiapi/oas_response_encoders_gen.go +++ b/internal/lokiapi/oas_response_encoders_gen.go @@ -13,6 +13,20 @@ import ( ht "github.com/ogen-go/ogen/http" ) +func encodeIndexStatsResponse(response *IndexStats, w http.ResponseWriter, span trace.Span) error { + w.Header().Set("Content-Type", "application/json; charset=utf-8") + w.WriteHeader(200) + span.SetStatus(codes.Ok, http.StatusText(200)) + + e := new(jx.Encoder) + response.Encode(e) + if _, err := e.WriteTo(w); err != nil { + return errors.Wrap(err, "write") + } + + return nil +} + func encodeLabelValuesResponse(response *Values, w http.ResponseWriter, span trace.Span) error { w.Header().Set("Content-Type", "application/json; charset=utf-8") w.WriteHeader(200) diff --git a/internal/lokiapi/oas_router_gen.go b/internal/lokiapi/oas_router_gen.go index 135f71ed..632e73ca 100644 --- a/internal/lokiapi/oas_router_gen.go +++ b/internal/lokiapi/oas_router_gen.go @@ -60,6 +60,24 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { break } switch elem[0] { + case 'i': // Prefix: "index/stats" + if l := len("index/stats"); len(elem) >= l && elem[0:l] == "index/stats" { + elem = elem[l:] + } else { + break + } + + if len(elem) == 0 { + // Leaf node. + switch r.Method { + case "GET": + s.handleIndexStatsRequest([0]string{}, elemIsEscaped, w, r) + default: + s.notAllowed(w, r, "GET") + } + + return + } case 'l': // Prefix: "label" if l := len("label"); len(elem) >= l && elem[0:l] == "label" { elem = elem[l:] @@ -296,6 +314,28 @@ func (s *Server) FindPath(method string, u *url.URL) (r Route, _ bool) { break } switch elem[0] { + case 'i': // Prefix: "index/stats" + if l := len("index/stats"); len(elem) >= l && elem[0:l] == "index/stats" { + elem = elem[l:] + } else { + break + } + + if len(elem) == 0 { + switch method { + case "GET": + // Leaf: IndexStats + r.name = "IndexStats" + r.summary = "" + r.operationID = "indexStats" + r.pathPattern = "/loki/api/v1/index/stats" + r.args = args + r.count = 0 + return r, true + default: + return + } + } case 'l': // Prefix: "label" if l := len("label"); len(elem) >= l && elem[0:l] == "label" { elem = elem[l:] diff --git a/internal/lokiapi/oas_schemas_gen.go b/internal/lokiapi/oas_schemas_gen.go index 0d8f5414..b1a32c1d 100644 --- a/internal/lokiapi/oas_schemas_gen.go +++ b/internal/lokiapi/oas_schemas_gen.go @@ -109,6 +109,54 @@ func (s *FPoint) SetV(val string) { s.V = val } +// Ref: #/components/schemas/IndexStats +type IndexStats struct { + Streams int `json:"streams"` + Chunks int `json:"chunks"` + Entries int `json:"entries"` + Bytes int `json:"bytes"` +} + +// GetStreams returns the value of Streams. +func (s *IndexStats) GetStreams() int { + return s.Streams +} + +// GetChunks returns the value of Chunks. +func (s *IndexStats) GetChunks() int { + return s.Chunks +} + +// GetEntries returns the value of Entries. +func (s *IndexStats) GetEntries() int { + return s.Entries +} + +// GetBytes returns the value of Bytes. +func (s *IndexStats) GetBytes() int { + return s.Bytes +} + +// SetStreams sets the value of Streams. +func (s *IndexStats) SetStreams(val int) { + s.Streams = val +} + +// SetChunks sets the value of Chunks. +func (s *IndexStats) SetChunks(val int) { + s.Chunks = val +} + +// SetEntries sets the value of Entries. +func (s *IndexStats) SetEntries(val int) { + s.Entries = val +} + +// SetBytes sets the value of Bytes. +func (s *IndexStats) SetBytes(val int) { + s.Bytes = val +} + // Ref: #/components/schemas/LabelSet type LabelSet map[string]string diff --git a/internal/lokiapi/oas_server_gen.go b/internal/lokiapi/oas_server_gen.go index 49288f8e..0ac83be3 100644 --- a/internal/lokiapi/oas_server_gen.go +++ b/internal/lokiapi/oas_server_gen.go @@ -8,6 +8,12 @@ import ( // Handler handles operations described by OpenAPI v3 specification. type Handler interface { + // IndexStats implements indexStats operation. + // + // Get index stats. + // + // GET /loki/api/v1/index/stats + IndexStats(ctx context.Context, params IndexStatsParams) (*IndexStats, error) // LabelValues implements labelValues operation. // // Get values of label. diff --git a/internal/lokiapi/oas_test_examples_gen_test.go b/internal/lokiapi/oas_test_examples_gen_test.go index b02fe2bf..52b72853 100644 --- a/internal/lokiapi/oas_test_examples_gen_test.go +++ b/internal/lokiapi/oas_test_examples_gen_test.go @@ -35,6 +35,18 @@ func TestFPoint_EncodeDecode(t *testing.T) { var typ2 FPoint require.NoError(t, typ2.Decode(jx.DecodeBytes(data))) } +func TestIndexStats_EncodeDecode(t *testing.T) { + var typ IndexStats + typ.SetFake() + + e := jx.Encoder{} + typ.Encode(&e) + data := e.Bytes() + require.True(t, std.Valid(data), "Encoded: %s", data) + + var typ2 IndexStats + require.NoError(t, typ2.Decode(jx.DecodeBytes(data))) +} func TestLabelSet_EncodeDecode(t *testing.T) { var typ LabelSet typ = make(LabelSet) diff --git a/internal/lokiapi/oas_unimplemented_gen.go b/internal/lokiapi/oas_unimplemented_gen.go index 7c7c805b..9a9db2d1 100644 --- a/internal/lokiapi/oas_unimplemented_gen.go +++ b/internal/lokiapi/oas_unimplemented_gen.go @@ -13,6 +13,15 @@ type UnimplementedHandler struct{} var _ Handler = UnimplementedHandler{} +// IndexStats implements indexStats operation. +// +// Get index stats. +// +// GET /loki/api/v1/index/stats +func (UnimplementedHandler) IndexStats(ctx context.Context, params IndexStatsParams) (r *IndexStats, _ error) { + return r, ht.ErrNotImplemented +} + // LabelValues implements labelValues operation. // // Get values of label. diff --git a/internal/lokihandler/lokihandler.go b/internal/lokihandler/lokihandler.go index 7b2bcd0a..4de480e3 100644 --- a/internal/lokihandler/lokihandler.go +++ b/internal/lokihandler/lokihandler.go @@ -27,6 +27,15 @@ type LokiAPI struct { engine *logqlengine.Engine } +// IndexStats implements indexStats operation. +// +// Get index stats. +// +// GET /loki/api/v1/index/stats +func (h *LokiAPI) IndexStats(ctx context.Context, params lokiapi.IndexStatsParams) (*lokiapi.IndexStats, error) { + return nil, ht.ErrNotImplemented +} + // NewLokiAPI creates new LokiAPI. func NewLokiAPI(q logstorage.Querier, engine *logqlengine.Engine) *LokiAPI { return &LokiAPI{ diff --git a/internal/lokiproxy/lokiproxy.go b/internal/lokiproxy/lokiproxy.go index 9356b65d..6a3a56a2 100644 --- a/internal/lokiproxy/lokiproxy.go +++ b/internal/lokiproxy/lokiproxy.go @@ -24,6 +24,15 @@ type Server struct { api *lokiapi.Client } +// IndexStats implements indexStats operation. +// +// Get index stats. +// +// GET /loki/api/v1/index/stats +func (s *Server) IndexStats(ctx context.Context, params lokiapi.IndexStatsParams) (*lokiapi.IndexStats, error) { + return s.api.IndexStats(ctx, params) +} + // LabelValues implements labelValues operation. // Get values of label. // From 9895cee8383138a292727c78b9987cbe700e7f21 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 29 Nov 2023 02:06:41 +0300 Subject: [PATCH 077/112] feat(chstorage): query summaries too --- internal/chstorage/querier_metrics.go | 138 ++++++++++++++++++++++++++ 1 file changed, 138 insertions(+) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 741f4c75..fb76c4f2 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -274,6 +274,7 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin points []storage.Series histSeries []storage.Series expHistSeries []storage.Series + summarySeries []storage.Series ) grp, grpCtx := errgroup.WithContext(ctx) grp.Go(func() error { @@ -321,12 +322,28 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin expHistSeries = result return nil }) + grp.Go(func() error { + ctx := grpCtx + + query, err := buildQuery(p.tables.Summaries) + if err != nil { + return err + } + + result, err := p.queryExpHistograms(ctx, query) + if err != nil { + return errors.Wrap(err, "query summaries") + } + summarySeries = result + return nil + }) if err := grp.Wait(); err != nil { return nil, err } points = append(points, histSeries...) points = append(points, expHistSeries...) + points = append(points, summarySeries...) return newSeriesSet(points), nil } @@ -626,6 +643,127 @@ func (p *promQuerier) queryExpHistograms(ctx context.Context, query string) ([]s return result, nil } +func (p *promQuerier) querySummaries(ctx context.Context, query string) ([]storage.Series, error) { + type seriesWithLabels struct { + series *series[pointData] + labels map[string]string + } + type summarySample struct { + timestamp int64 + rawAttributes, rawResource string + attributes, resource map[string]string + flags pmetric.DataPointFlags + } + + var ( + set = map[seriesKey]seriesWithLabels{} + addSample = func( + name string, + val float64, + sample summarySample, + bucketKey [2]string, + ) { + key := seriesKey{ + name: name, + attributes: sample.rawAttributes, + resource: sample.rawResource, + bucketKey: bucketKey, + } + s, ok := set[key] + if !ok { + s = seriesWithLabels{ + series: &series[pointData]{}, + labels: map[string]string{}, + } + set[key] = s + } + + if sample.flags.NoRecordedValue() { + val = math.Float64frombits(value.StaleNaN) + } + s.series.data.values = append(s.series.data.values, val) + s.series.ts = append(s.series.ts, sample.timestamp) + + s.labels["__name__"] = name + maps.Copy(s.labels, sample.attributes) + maps.Copy(s.labels, sample.resource) + if key := bucketKey[0]; key != "" { + s.labels[key] = bucketKey[1] + } + } + c = newSummaryColumns() + ) + if err := p.ch.Do(ctx, ch.Query{ + Body: query, + Result: c.Result(), + OnResult: func(ctx context.Context, block proto.Block) error { + for i := 0; i < c.timestamp.Rows(); i++ { + name := c.name.Row(i) + timestamp := c.timestamp.Row(i) + count := c.count.Row(i) + sum := c.sum.Row(i) + quantiles := c.quantiles.Row(i) + values := c.values.Row(i) + flags := pmetric.DataPointFlags(c.flags.Row(i)) + rawAttributes := c.attributes.Row(i) + rawResource := c.resource.Row(i) + + var ( + resource = map[string]string{} + attributes = map[string]string{} + ) + if err := parseLabels(rawResource, resource); err != nil { + return errors.Wrap(err, "parse resource") + } + if err := parseLabels(rawAttributes, attributes); err != nil { + return errors.Wrap(err, "parse attributes") + } + sample := summarySample{ + timestamp: timestamp.UnixMilli(), + rawAttributes: rawAttributes, + rawResource: rawResource, + attributes: attributes, + resource: resource, + flags: flags, + } + + addSample(name+"_count", float64(count), sample, [2]string{}) + addSample(name+"_sum", sum, sample, [2]string{}) + + for i := 0; i < min(len(quantiles), len(values)); i++ { + quantile := quantiles[i] + value := values[i] + + // Generate series with "quantile" label. + addSample("", value, sample, [2]string{ + "quantile", + strconv.FormatFloat(quantile, 'f', -1, 64), + }) + } + } + return nil + }, + }); err != nil { + return nil, errors.Wrap(err, "do query") + } + + var ( + result = make([]storage.Series, 0, len(set)) + lb labels.ScratchBuilder + ) + for _, s := range set { + lb.Reset() + for key, value := range s.labels { + lb.Add(key, value) + } + lb.Sort() + s.series.labels = lb.Labels() + result = append(result, s.series) + } + + return result, nil +} + type seriesSet[S storage.Series] struct { set []S n int From 240bfde5f4c07e253f0067ad77eaab40d1e97e0e Mon Sep 17 00:00:00 2001 From: tdakkota Date: Wed, 29 Nov 2023 04:53:23 +0300 Subject: [PATCH 078/112] fix(chstorage): call the right function --- internal/chstorage/querier_metrics.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index fb76c4f2..6f1dda23 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -330,7 +330,7 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin return err } - result, err := p.queryExpHistograms(ctx, query) + result, err := p.querySummaries(ctx, query) if err != nil { return errors.Wrap(err, "query summaries") } From 07718ef0e3a6fa49e3ddcb7173f3c8e486f950e0 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 15:19:48 +0300 Subject: [PATCH 079/112] feat(otelfaker): improve --- cmd/otelfaker/main.go | 50 +++++++++++++++++++++++++++---------------- 1 file changed, 31 insertions(+), 19 deletions(-) diff --git a/cmd/otelfaker/main.go b/cmd/otelfaker/main.go index c1e3791a..48877d4d 100644 --- a/cmd/otelfaker/main.go +++ b/cmd/otelfaker/main.go @@ -26,38 +26,50 @@ func getLogs(ctx context.Context, tracer trace.Tracer, rnd *rand.Rand, now time. spanContext = span.SpanContext() traceID = spanContext.TraceID() spanID = spanContext.SpanID() + ld = plog.NewLogs() + rl = ld.ResourceLogs().AppendEmpty() ) - ld := plog.NewLogs() - rl := ld.ResourceLogs().AppendEmpty() - rl.Resource().Attributes().PutStr("host.name", "testHost") + { + a := rl.Resource().Attributes() + a.PutStr("service.name", "testService") + a.PutStr("service.version", "1.0.0") + a.PutStr("service.instance.id", "testInstance") + a.PutStr("service.namespace", "testNamespace") + a.PutStr("host.name", "testHost") + } rl.SetSchemaUrl("resource_schema") il := rl.ScopeLogs().AppendEmpty() - il.Scope().SetName("name") - il.Scope().SetVersion("version") - il.Scope().Attributes().PutStr("oteldb.name", "testDB") - il.Scope().SetDroppedAttributesCount(1) + { + s := il.Scope() + s.SetName("name") + s.SetVersion("version") + s.Attributes().PutStr("oteldb.name", "testDB") + s.SetDroppedAttributesCount(1) + } il.SetSchemaUrl("scope_schema") lg := il.LogRecords().AppendEmpty() lg.SetSeverityNumber(plog.SeverityNumber(logspb.SeverityNumber_SEVERITY_NUMBER_INFO)) lg.SetSeverityText("Info") - lg.SetFlags(plog.LogRecordFlags(logspb.LogRecordFlags_LOG_RECORD_FLAGS_DO_NOT_USE)) lg.SetTraceID(pcommon.TraceID(traceID)) lg.SetSpanID(pcommon.SpanID(spanID)) lg.Body().SetStr("hello world") lg.SetTimestamp(pcommon.Timestamp(now.UnixNano())) lg.SetObservedTimestamp(pcommon.Timestamp(now.UnixNano())) - lg.Attributes().PutStr("sdkVersion", "1.0.1") - lg.Attributes().PutStr("http.method", "GET") - lg.Attributes().PutBool("http.server", true) - lg.Attributes().PutInt("http.status_code", 200) - if rnd.Float32() < 0.5 { - lg.Attributes().PutStr("http.url", "https://example.com") - lg.Attributes().PutStr("http.status_text", "OK") - } else { - lg.Attributes().PutStr("http.user_agent", "test-agent") + { + a := lg.Attributes() + a.PutStr("sdkVersion", "1.0.1") + a.PutStr("http.method", "GET") + a.PutBool("http.server", true) + a.PutInt("http.status_code", 200) + if rnd.Float32() < 0.5 { + a.PutStr("http.url", "https://example.com") + a.PutStr("http.status_text", "OK") + } else { + a.PutStr("http.user_agent", "test-agent") + } + a.PutDouble("http.duration_seconds", 1.1054) + a.PutInt("http.duration", (time.Second + time.Millisecond*105).Nanoseconds()) } - lg.Attributes().PutDouble("http.duration_seconds", 1.1054) - lg.Attributes().PutInt("http.duration", (time.Second + time.Millisecond*105).Nanoseconds()) lg.SetFlags(plog.DefaultLogRecordFlags.WithIsSampled(true)) return ld } From 8295bb5761834ba39c069578857c206e85d07b24 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 15:19:54 +0300 Subject: [PATCH 080/112] docs(logs): init --- docs/logs.md | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 docs/logs.md diff --git a/docs/logs.md b/docs/logs.md new file mode 100644 index 00000000..5e44ca2d --- /dev/null +++ b/docs/logs.md @@ -0,0 +1,34 @@ +# Logs model + +## OpenTelemetry + +The OpenTelemetry log model can be marshalled as JSON entry: + +```json +{ + "body": "hello world", + "traceid": "cacb16ac5ace52ebe8749114234a8e58", + "spanid": "36a431d0481b2744", + "severity": "Info", + "flags": 1, + "attributes": { + "http.duration": 1105000000, + "http.duration_seconds": 1.1054, + "http.method": "GET", + "http.server": true, + "http.status_code": 200, + "http.user_agent": "test-agent", + "sdkVersion": "1.0.1" + }, + "resources": { + "host.name": "testHost" + }, + "instrumentation_scope": { + "name": "name", + "version": "version", + "attributes": { + "oteldb.name": "testDB" + } + } +} +``` \ No newline at end of file From 5a3d9c231736f4835688ed5ee9f673d9d6428364 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 15:57:34 +0300 Subject: [PATCH 081/112] feat(dev.local.ch): add tempo --- dev/local/ch/docker-compose.yml | 10 +++- .../{datasources.yaml => datasources.yml} | 13 ++++- dev/local/otelcol.yml | 6 ++- dev/local/tempo.yml | 47 +++++++++++++++++++ 4 files changed, 73 insertions(+), 3 deletions(-) rename dev/local/grafana/{datasources.yaml => datasources.yml} (92%) create mode 100644 dev/local/tempo.yml diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index 5f2d873d..602e0fe6 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -2,6 +2,7 @@ version: "3" volumes: prom: + tempo: services: clickhouse: @@ -172,6 +173,13 @@ services: jaeger: image: jaegertracing/all-in-one:latest + tempo: + image: grafana/tempo:latest + command: [ "-config.file=/etc/tempo.yml" ] + volumes: + - ../tempo.yml:/etc/tempo.yml + - tempo:/tmp/tempo + grafana: image: "grafana/grafana:10.0.0" environment: @@ -187,7 +195,7 @@ services: - "3000:3000" volumes: - ../grafana/grafana.ini:/etc/grafana/grafana.ini - - ../grafana/datasources.yaml:/etc/grafana/provisioning/datasources/datasources.yaml + - ../grafana/datasources.yml:/etc/grafana/provisioning/datasources/datasources.yml - ../grafana/dashboards.yml:/etc/grafana/provisioning/dashboards/default.yml - ../grafana/dashboards:/etc/grafana/dashboards:ro depends_on: diff --git a/dev/local/grafana/datasources.yaml b/dev/local/grafana/datasources.yml similarity index 92% rename from dev/local/grafana/datasources.yaml rename to dev/local/grafana/datasources.yml index 7dd69903..c0a78750 100644 --- a/dev/local/grafana/datasources.yaml +++ b/dev/local/grafana/datasources.yml @@ -6,7 +6,7 @@ datasources: access: proxy orgId: 1 url: http://oteldb:3200 - uid: tempo + uid: tempo-oteldb jsonData: httpMethod: GET serviceMap: @@ -62,6 +62,17 @@ datasources: url: http://jaeger:16686 access: proxy + - name: Tempo + type: tempo + uid: tempo + url: http://tempo:3200 + access: proxy + httpMethod: GET + jsonData: + httpMethod: GET + serviceMap: + datasourceUid: prometheus + # For qryn - name: "qryn: LogQL" type: loki diff --git a/dev/local/otelcol.yml b/dev/local/otelcol.yml index 3b0cd3a7..99b80a14 100644 --- a/dev/local/otelcol.yml +++ b/dev/local/otelcol.yml @@ -61,6 +61,10 @@ exporters: endpoint: jaeger:4317 tls: insecure: true + otlp/tempo: + endpoint: tempo:4317 + tls: + insecure: true otlp/qryn: endpoint: qryn-collector:4317 tls: @@ -91,7 +95,7 @@ service: traces: receivers: [otlp] processors: [batch] - exporters: [otlp, otlp/jaeger, otlp/qryn] + exporters: [otlp, otlp/jaeger, otlp/qryn, otlp/tempo] metrics: receivers: [otlp, prometheus, hostmetrics] processors: [batch] diff --git a/dev/local/tempo.yml b/dev/local/tempo.yml new file mode 100644 index 00000000..2eb2e887 --- /dev/null +++ b/dev/local/tempo.yml @@ -0,0 +1,47 @@ +server: + http_listen_port: 3200 + +query_frontend: + search: + duration_slo: 5s + throughput_bytes_slo: 1.073741824e+09 + trace_by_id: + duration_slo: 5s + +distributor: + receivers: + otlp: + protocols: + http: + grpc: + +ingester: + max_block_duration: 5m # cut the headblock when this much time passes. this is being set for demo purposes and should probably be left alone normally + +compactor: + compaction: + block_retention: 1h # overall Tempo trace retention. set for demo purposes + +metrics_generator: + registry: + external_labels: + source: tempo + cluster: docker-compose + storage: + path: /tmp/tempo/generator/wal + remote_write: + - url: http://prometheus:9090/api/v1/write + send_exemplars: true + +storage: + trace: + backend: local # backend configuration to use + wal: + path: /tmp/tempo/wal # where to store the the wal locally + local: + path: /tmp/tempo/blocks + +overrides: + defaults: + metrics_generator: + processors: [service-graphs, span-metrics] # enables metrics generator \ No newline at end of file From 6a2c4c771bcea64a0d5b5504ce8f0755a90f6adf Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 16:09:57 +0300 Subject: [PATCH 082/112] feat(chotel): set span kind --- cmd/chotel/main.go | 13 ++++++++++--- internal/chtrace/chtrace.go | 17 +++++++++++++++++ 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/cmd/chotel/main.go b/cmd/chotel/main.go index 55771e5e..12205d68 100644 --- a/cmd/chotel/main.go +++ b/cmd/chotel/main.go @@ -182,6 +182,14 @@ func (a *App) send(ctx context.Context, now time.Time) error { ExportedAt proto.ColDateTime } ) + clickhouseResource, err := resource.New(ctx, + resource.WithAttributes( + semconv.ServiceNameKey.String("clickhouse"), + ), + ) + if err != nil { + return errors.Wrap(err, "clickhouse resource") + } var latest time.Time if err := db.Do(ctx, ch.Query{ Body: q, @@ -192,9 +200,8 @@ func (a *App) send(ctx context.Context, now time.Time) error { for _, r := range t.Rows() { exported.ExportedAt.Append(now) stub := tracetest.SpanStub{ - Resource: resource.NewSchemaless( - semconv.ServiceNameKey.String("clickhouse"), - ), + SpanKind: r.Kind, + Resource: clickhouseResource, Name: r.OperationName, StartTime: r.StartTime, EndTime: r.FinishTime, diff --git a/internal/chtrace/chtrace.go b/internal/chtrace/chtrace.go index d9589379..4418ea6e 100644 --- a/internal/chtrace/chtrace.go +++ b/internal/chtrace/chtrace.go @@ -19,6 +19,7 @@ type Table struct { FinishTimeMicro proto.ColUInt64 // finish_time_us FinishDate proto.ColDate // finish_date Attributes proto.ColMap[string, string] // attribute + SpanKind proto.ColEnum // span_kind } // Rows returns Trace per row. @@ -32,6 +33,20 @@ func (t Table) Rows() []Trace { Attributes: t.Attributes.Row(i), OperationName: t.OperationName.Row(i), } + switch t.SpanKind.Row(i) { + case "SERVER": + tt.Kind = trace.SpanKindServer + case "CLIENT": + tt.Kind = trace.SpanKindClient + case "INTERNAL": + tt.Kind = trace.SpanKindInternal + case "PRODUCER": + tt.Kind = trace.SpanKindProducer + case "CONSUMER": + tt.Kind = trace.SpanKindConsumer + default: + tt.Kind = trace.SpanKindInternal + } binary.BigEndian.PutUint64(tt.SpanID[:], t.SpanID.Row(i)) binary.BigEndian.PutUint64(tt.ParentSpanID[:], t.ParentSpanID.Row(i)) @@ -50,6 +65,7 @@ type Trace struct { StartTime time.Time FinishTime time.Time Attributes map[string]string + Kind trace.SpanKind } // Result returns proto.Results for Table. @@ -63,6 +79,7 @@ func (t *Table) Result() proto.Results { {Name: "finish_time_us", Data: &t.FinishTimeMicro}, {Name: "finish_date", Data: &t.FinishDate}, {Name: "attribute", Data: &t.Attributes}, + {Name: "kind", Data: &t.SpanKind}, } } From 557e712c7c224bc546d2904b16c183ab557df0b5 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 16:30:42 +0300 Subject: [PATCH 083/112] feat(otelfaker): add multiple levels --- cmd/otelfaker/main.go | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/cmd/otelfaker/main.go b/cmd/otelfaker/main.go index 48877d4d..e030dcaa 100644 --- a/cmd/otelfaker/main.go +++ b/cmd/otelfaker/main.go @@ -48,8 +48,23 @@ func getLogs(ctx context.Context, tracer trace.Tracer, rnd *rand.Rand, now time. } il.SetSchemaUrl("scope_schema") lg := il.LogRecords().AppendEmpty() - lg.SetSeverityNumber(plog.SeverityNumber(logspb.SeverityNumber_SEVERITY_NUMBER_INFO)) - lg.SetSeverityText("Info") + switch { + case rnd.Float32() < 0.1: + lg.SetSeverityNumber(plog.SeverityNumber(logspb.SeverityNumber_SEVERITY_NUMBER_DEBUG)) + lg.SetSeverityText("Debug") + case rnd.Float32() < 0.2: + lg.SetSeverityNumber(plog.SeverityNumber(logspb.SeverityNumber_SEVERITY_NUMBER_ERROR)) + lg.SetSeverityText("Error") + case rnd.Float32() < 0.3: + lg.SetSeverityNumber(plog.SeverityNumber(logspb.SeverityNumber_SEVERITY_NUMBER_FATAL)) + lg.SetSeverityText("Fatal") + case rnd.Float32() < 0.4: + lg.SetSeverityNumber(plog.SeverityNumber(logspb.SeverityNumber_SEVERITY_NUMBER_WARN)) + lg.SetSeverityText("Warn") + default: + lg.SetSeverityNumber(plog.SeverityNumber(logspb.SeverityNumber_SEVERITY_NUMBER_INFO)) + lg.SetSeverityText("Info") + } lg.SetTraceID(pcommon.TraceID(traceID)) lg.SetSpanID(pcommon.SpanID(spanID)) lg.Body().SetStr("hello world") From d2e1a422f65d076de83fd85c18a7331239f53086 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 20:02:37 +0300 Subject: [PATCH 084/112] docs(logs): upd --- docs/logs.md | 100 +++++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 98 insertions(+), 2 deletions(-) diff --git a/docs/logs.md b/docs/logs.md index 5e44ca2d..38c64569 100644 --- a/docs/logs.md +++ b/docs/logs.md @@ -2,7 +2,7 @@ ## OpenTelemetry -The OpenTelemetry log model can be marshalled as JSON entry: +The OpenTelemetry log model can be marshalled as JSON document: ```json { @@ -31,4 +31,100 @@ The OpenTelemetry log model can be marshalled as JSON entry: } } } -``` \ No newline at end of file +``` + +## LogQL + +The LogQL unlike TraceQL inherits Prometheus label model constraints, i.e. no dots in label names +and label value is only string. + +Using example above, lets explore our options for mapping. + +### Resource attributes + +Resource attributes can be straightforwardly mapped to labels: + +``` +otel_logs{host_name="testHost"} +``` + +Dots are replaced with underscores. + +> [!WARNING] +> Mapping is not reversible, so should be persisted. + +Optionally, we can prefix them with `resource_`. + +### Attributes + +It is unclear whether attributes should be mapped to labels. +On one hand, it is possible to do so: + +``` +{http_method="GET"} +``` + +Optionally with `attribute_` prefix. + +On the other hand, it is desirable to make more complex queries, e.g.: + +``` +{service_name="testService"} | json | attributes_http_duration > 30s or attributes_status_code != 200 +``` + +This approach raises multiple questions: +1. Should we really use `attributes_` prefix? +2. If not, should we search both in attributes, resource and scope? +3. How to deal with grafana auto-detection of labels? Should we change json representation? + +### Line filter expression + +``` +{job="testNamespace/testService"} |= `hello world` +``` + +1. Should we search only in body? +2. Should we search in all fields? +3. How to deal with traceID and spanID when using trace-to-logs? + + +## Draft solution + +Log message: + +```json +{ + "body": "hello world", + "trace_id": "cacb16ac5ace52ebe8749114234a8e58", + "span_id": "36a431d0481b2744", + "severity": "Info", + "http.duration": 1105000000, + "http.duration.seconds": 1.1054, + "http.method": "GET", + "http.server": true, + "http.status_code": 200, + "http.user_agent": "test-agent", +} +``` + +Labels: + +``` +severity="INFO" (should be from severity level as per standard) +service_name="testService" +service_namespace="testNamespace" + +// Not sure about this one, will require storing mapping to support regex +job="testNamespace/testService" ( + +// Optionally: +http_method="GET" +http_server="true" +http_status_code="200" +``` + +Log line search: everywhere except resource attributes and scope. +Special case for traceID and spanID with additional search in traceID and spanID fields. + +Labels extraction: handle first `| json |` explicitly, offloading to underlying log storage +where reverse mapping should be executed. \ No newline at end of file From 1f495508c90dd93b259c3133d5ae9323e0352278 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 20:13:10 +0300 Subject: [PATCH 085/112] docs: upd --- docs/logs.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/logs.md b/docs/logs.md index 38c64569..839012ba 100644 --- a/docs/logs.md +++ b/docs/logs.md @@ -115,7 +115,7 @@ service_name="testService" service_namespace="testNamespace" // Not sure about this one, will require storing mapping to support regex -job="testNamespace/testService" ( +job="testNamespace/testService". // Optionally: http_method="GET" @@ -123,6 +123,8 @@ http_server="true" http_status_code="200" ``` +This will allow filtering by this labels in Grafana UI by clicking on it. + Log line search: everywhere except resource attributes and scope. Special case for traceID and spanID with additional search in traceID and spanID fields. From 87886ea216dc4638127eea2023849f73d888ce13 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 20:22:28 +0300 Subject: [PATCH 086/112] docs(logs): don't search in all attrs --- docs/logs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/logs.md b/docs/logs.md index 839012ba..046b1934 100644 --- a/docs/logs.md +++ b/docs/logs.md @@ -125,7 +125,7 @@ http_status_code="200" This will allow filtering by this labels in Grafana UI by clicking on it. -Log line search: everywhere except resource attributes and scope. +Log line search: only in body field. Special case for traceID and spanID with additional search in traceID and spanID fields. Labels extraction: handle first `| json |` explicitly, offloading to underlying log storage From 0f1c1e9a353049ecaa26674e11c8377ed723f546 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 20:30:37 +0300 Subject: [PATCH 087/112] docs(logs): describe extracted labels --- docs/logs.md | 26 ++++++++++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/docs/logs.md b/docs/logs.md index 046b1934..d70191b5 100644 --- a/docs/logs.md +++ b/docs/logs.md @@ -97,7 +97,7 @@ Log message: "body": "hello world", "trace_id": "cacb16ac5ace52ebe8749114234a8e58", "span_id": "36a431d0481b2744", - "severity": "Info", + "severity": "INFO", "http.duration": 1105000000, "http.duration.seconds": 1.1054, "http.method": "GET", @@ -129,4 +129,26 @@ Log line search: only in body field. Special case for traceID and spanID with additional search in traceID and spanID fields. Labels extraction: handle first `| json |` explicitly, offloading to underlying log storage -where reverse mapping should be executed. \ No newline at end of file +where reverse mapping should be executed. + +Extracted labels after `json`: +``` +body: "hello world" +trace_id="cacb16ac5ace52ebe8749114234a8e58" +span_id="36a431d0481b2744" +severity="INFO" +http_duration=1105000000 +http_duration_seconds=1.1054 +http_method="GET" +http_server=true +http_status_code=200 +http_user_agent=test-agent +``` + +So we can have a query like this: + +``` +{service_name="testService"} ~= `hello world` | json | http_duration > 30s or http_status_code >= 500 +``` + +Which can be translated to efficient ClickHouse query. \ No newline at end of file From 626c308a55f314749289a7c5b3d39455d89d7cc7 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Wed, 29 Nov 2023 20:32:31 +0300 Subject: [PATCH 088/112] docs(logs): typo --- docs/logs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/logs.md b/docs/logs.md index d70191b5..e65a5afc 100644 --- a/docs/logs.md +++ b/docs/logs.md @@ -133,7 +133,7 @@ where reverse mapping should be executed. Extracted labels after `json`: ``` -body: "hello world" +body="hello world" trace_id="cacb16ac5ace52ebe8749114234a8e58" span_id="36a431d0481b2744" severity="INFO" From 5f0e4183a2cf64dc8a80ee7bed35e9ee22a90061 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Thu, 30 Nov 2023 04:26:42 +0300 Subject: [PATCH 089/112] fix(chstorage): dedup labels before inserting --- internal/chstorage/inserter_metrics.go | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index fa4e142b..7aefce27 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -29,7 +29,7 @@ type metricsBatch struct { histograms *histogramColumns expHistograms *expHistogramColumns summaries *summaryColumns - labels *labelsColumns + labels map[[2]string]struct{} } func newMetricBatch() *metricsBatch { @@ -38,13 +38,18 @@ func newMetricBatch() *metricsBatch { histograms: newHistogramColumns(), expHistograms: newExpHistogramColumns(), summaries: newSummaryColumns(), - labels: newLabelsColumns(), + labels: map[[2]string]struct{}{}, } } func (b *metricsBatch) Insert(ctx context.Context, tables Tables, client *chpool.Pool) error { - grp, grpCtx := errgroup.WithContext(ctx) + labels := newLabelsColumns() + for pair := range b.labels { + labels.name.Append(pair[0]) + labels.value.Append(pair[1]) + } + grp, grpCtx := errgroup.WithContext(ctx) type columns interface { Input() proto.Input } @@ -56,7 +61,7 @@ func (b *metricsBatch) Insert(ctx context.Context, tables Tables, client *chpool {tables.Histograms, b.histograms}, {tables.ExpHistograms, b.expHistograms}, {tables.Summaries, b.summaries}, - {tables.Labels, b.labels}, + {tables.Labels, labels}, } { table := table grp.Go(func() error { @@ -242,10 +247,13 @@ func (b *metricsBatch) addSummaryPoints(name string, res pcommon.Map, slice pmet } func (b *metricsBatch) addLabels(m pcommon.Map) { - m.Range(func(k string, v pcommon.Value) bool { - b.labels.name.Append(k) - // FIXME(tdakkota): annoying allocations - b.labels.value.Append(v.AsString()) + m.Range(func(name string, value pcommon.Value) bool { + key := [2]string{ + name, + // FIXME(tdakkota): annoying allocations + value.AsString(), + } + b.labels[key] = struct{}{} return true }) } From 33c5c4675c1b048ab326f6dd9c28a208c9e146cd Mon Sep 17 00:00:00 2001 From: tdakkota Date: Thu, 30 Nov 2023 04:57:40 +0300 Subject: [PATCH 090/112] refactor(chstorage): map histograms and summaries into points --- internal/chstorage/columns_metrics.go | 5 +- internal/chstorage/inserter_metrics.go | 129 ++++++++++- internal/chstorage/querier_metrics.go | 296 ------------------------- internal/chstorage/schema_metrics.go | 26 +++ internal/chstorage/schema_traces.go | 12 +- 5 files changed, 163 insertions(+), 305 deletions(-) diff --git a/internal/chstorage/columns_metrics.go b/internal/chstorage/columns_metrics.go index 9396b6f8..4cb8c2aa 100644 --- a/internal/chstorage/columns_metrics.go +++ b/internal/chstorage/columns_metrics.go @@ -8,7 +8,8 @@ type pointColumns struct { name *proto.ColLowCardinality[string] timestamp *proto.ColDateTime64 - value proto.ColFloat64 + mapping proto.ColEnum8 + value proto.ColFloat64 flags proto.ColUInt32 attributes proto.ColStr @@ -27,6 +28,7 @@ func (c *pointColumns) Input() proto.Input { {Name: "name", Data: c.name}, {Name: "timestamp", Data: c.timestamp}, + {Name: "mapping", Data: proto.Wrap(&c.mapping, metricMappingDDL)}, {Name: "value", Data: c.value}, {Name: "flags", Data: c.flags}, @@ -41,6 +43,7 @@ func (c *pointColumns) Result() proto.Results { {Name: "name", Data: c.name}, {Name: "timestamp", Data: c.timestamp}, + {Name: "mapping", Data: &c.mapping}, {Name: "value", Data: &c.value}, {Name: "flags", Data: &c.flags}, diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index 7aefce27..3aded365 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -2,6 +2,8 @@ package chstorage import ( "context" + "strconv" + "time" "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/chpool" @@ -110,6 +112,7 @@ func (b *metricsBatch) addPoints(name string, res pcommon.Map, slice pmetric.Num b.addLabels(attrs) c.name.Append(name) c.timestamp.Append(ts) + c.mapping.Append(proto.Enum8(noMapping)) c.value.Append(val) c.flags.Append(uint32(flags)) c.attributes.Append(encodeAttributes(attrs)) @@ -130,11 +133,11 @@ func (b *metricsBatch) addHistogramPoints(name string, res pcommon.Map, slice pm Set: point.HasSum(), Value: point.Sum(), } - min := proto.Nullable[float64]{ + _min := proto.Nullable[float64]{ Set: point.HasMin(), Value: point.Min(), } - max := proto.Nullable[float64]{ + _max := proto.Nullable[float64]{ Set: point.HasMax(), Value: point.Max(), } @@ -142,17 +145,92 @@ func (b *metricsBatch) addHistogramPoints(name string, res pcommon.Map, slice pm explicitBounds := point.ExplicitBounds().AsRaw() b.addLabels(attrs) + // Save original histogram. c.name.Append(name) c.timestamp.Append(ts) c.count.Append(count) c.sum.Append(sum) - c.min.Append(min) - c.max.Append(max) + c.min.Append(_min) + c.max.Append(_max) c.bucketCounts.Append(bucketCounts) c.explicitBounds.Append(explicitBounds) c.flags.Append(uint32(flags)) c.attributes.Append(encodeAttributes(attrs)) c.resource.Append(encodeAttributes(res)) + + // Map histogram as set of series for Prometheus compatibility. + series := mappedSeries{ + ts: ts, + flags: flags, + attrs: attrs, + res: res, + } + if sum.Set { + b.addMappedSample( + series, + name+"_sum", + histogramSum, + sum.Value, + [2]string{}, + ) + } + if _min.Set { + b.addMappedSample( + series, + name+"_min", + histogramMin, + _min.Value, + [2]string{}, + ) + } + if _max.Set { + b.addMappedSample( + series, + name+"_max", + histogramMax, + _max.Value, + [2]string{}, + ) + } + b.addMappedSample( + series, + name+"_count", + histogramCount, + float64(count), + [2]string{}, + ) + + var ( + cumCount uint64 + bucketName = name + "_bucket" + ) + for i := 0; i < min(len(bucketCounts), len(explicitBounds)); i++ { + bound := explicitBounds[i] + cumCount += bucketCounts[i] + + // Generate series with "_bucket" suffix and "le" label. + b.addMappedSample( + series, + bucketName, + histogramBucket, + float64(cumCount), + [2]string{ + "le", + strconv.FormatFloat(bound, 'f', -1, 64), + }, + ) + } + // Generate series with "_bucket" suffix and "le" label. + b.addMappedSample( + series, + bucketName, + histogramBucket, + float64(cumCount), + [2]string{ + "le", + "+Inf", + }, + ) } return nil } @@ -242,10 +320,53 @@ func (b *metricsBatch) addSummaryPoints(name string, res pcommon.Map, slice pmet c.flags.Append(uint32(flags)) c.attributes.Append(encodeAttributes(attrs)) c.resource.Append(encodeAttributes(res)) + + series := mappedSeries{ + ts: ts, + flags: flags, + attrs: attrs, + res: res, + } + b.addMappedSample(series, name+"_count", summaryCount, float64(count), [2]string{}) + b.addMappedSample(series, name+"_sum", summarySum, sum, [2]string{}) + + for i := 0; i < min(len(quantiles), len(values)); i++ { + quantile := quantiles[i] + value := values[i] + + // Generate series with "quantile" label. + b.addMappedSample(series, name, summaryQuantile, value, [2]string{ + "quantile", + strconv.FormatFloat(quantile, 'f', -1, 64), + }) + } } return nil } +type mappedSeries struct { + ts time.Time + flags pmetric.DataPointFlags + attrs, res pcommon.Map +} + +func (b *metricsBatch) addMappedSample( + series mappedSeries, + name string, + mapping metricMapping, + val float64, + bucketKey [2]string, +) { + c := b.points + c.name.Append(name) + c.timestamp.Append(series.ts) + c.mapping.Append(proto.Enum8(mapping)) + c.value.Append(val) + c.flags.Append(uint32(series.flags)) + c.attributes.Append(encodeAttributes(series.attrs, bucketKey)) + c.resource.Append(encodeAttributes(series.res, bucketKey)) +} + func (b *metricsBatch) addLabels(m pcommon.Map) { m.Range(func(name string, value pcommon.Value) bool { key := [2]string{ diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 6f1dda23..17d32226 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -3,9 +3,6 @@ package chstorage import ( "context" "fmt" - "maps" - "math" - "strconv" "strings" "time" @@ -13,11 +10,9 @@ import ( "github.com/ClickHouse/ch-go/proto" "github.com/go-faster/errors" "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/model/value" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/annotations" - "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" "golang.org/x/sync/errgroup" @@ -295,21 +290,6 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin grp.Go(func() error { ctx := grpCtx - query, err := buildQuery(p.tables.Histograms) - if err != nil { - return err - } - - result, err := p.queryHistograms(ctx, query) - if err != nil { - return errors.Wrap(err, "query histograms") - } - histSeries = result - return nil - }) - grp.Go(func() error { - ctx := grpCtx - query, err := buildQuery(p.tables.ExpHistograms) if err != nil { return err @@ -322,21 +302,6 @@ func (p *promQuerier) selectSeries(ctx context.Context, hints *storage.SelectHin expHistSeries = result return nil }) - grp.Go(func() error { - ctx := grpCtx - - query, err := buildQuery(p.tables.Summaries) - if err != nil { - return err - } - - result, err := p.querySummaries(ctx, query) - if err != nil { - return errors.Wrap(err, "query summaries") - } - summarySeries = result - return nil - }) if err := grp.Wait(); err != nil { return nil, err } @@ -416,146 +381,6 @@ func (p *promQuerier) queryPoints(ctx context.Context, query string) ([]storage. return result, nil } -func (p *promQuerier) queryHistograms(ctx context.Context, query string) ([]storage.Series, error) { - type seriesWithLabels struct { - series *series[pointData] - labels map[string]string - } - type histogramSample struct { - timestamp int64 - rawAttributes, rawResource string - attributes, resource map[string]string - flags pmetric.DataPointFlags - } - - var ( - set = map[seriesKey]seriesWithLabels{} - addSample = func( - name string, - val float64, - sample histogramSample, - bucketKey [2]string, - ) { - key := seriesKey{ - name: name, - attributes: sample.rawAttributes, - resource: sample.rawResource, - bucketKey: bucketKey, - } - s, ok := set[key] - if !ok { - s = seriesWithLabels{ - series: &series[pointData]{}, - labels: map[string]string{}, - } - set[key] = s - } - - if sample.flags.NoRecordedValue() { - val = math.Float64frombits(value.StaleNaN) - } - s.series.data.values = append(s.series.data.values, val) - s.series.ts = append(s.series.ts, sample.timestamp) - - s.labels["__name__"] = name - maps.Copy(s.labels, sample.attributes) - maps.Copy(s.labels, sample.resource) - if key := bucketKey[0]; key != "" { - s.labels[key] = bucketKey[1] - } - } - c = newHistogramColumns() - ) - if err := p.ch.Do(ctx, ch.Query{ - Body: query, - Result: c.Result(), - OnResult: func(ctx context.Context, block proto.Block) error { - for i := 0; i < c.timestamp.Rows(); i++ { - name := c.name.Row(i) - timestamp := c.timestamp.Row(i) - count := c.count.Row(i) - sum := c.sum.Row(i) - _min := c.min.Row(i) - _max := c.max.Row(i) - bucketCounts := c.bucketCounts.Row(i) - explicitBounds := c.explicitBounds.Row(i) - flags := pmetric.DataPointFlags(c.flags.Row(i)) - rawAttributes := c.attributes.Row(i) - rawResource := c.resource.Row(i) - - var ( - resource = map[string]string{} - attributes = map[string]string{} - ) - if err := parseLabels(rawResource, resource); err != nil { - return errors.Wrap(err, "parse resource") - } - if err := parseLabels(rawAttributes, attributes); err != nil { - return errors.Wrap(err, "parse attributes") - } - sample := histogramSample{ - timestamp: timestamp.UnixMilli(), - rawAttributes: rawAttributes, - rawResource: rawResource, - attributes: attributes, - resource: resource, - flags: flags, - } - - if sum.Set { - addSample(name+"_sum", sum.Value, sample, [2]string{}) - } - if _min.Set { - addSample(name+"_min", _min.Value, sample, [2]string{}) - } - if _max.Set { - addSample(name+"_max", _max.Value, sample, [2]string{}) - } - addSample(name+"_count", float64(count), sample, [2]string{}) - - var cumCount uint64 - for i := 0; i < min(len(bucketCounts), len(explicitBounds)); i++ { - bound := explicitBounds[i] - cumCount += bucketCounts[i] - - // Generate series with "_bucket" suffix and "le" label. - addSample("_bucket", float64(cumCount), sample, [2]string{ - "le", - strconv.FormatFloat(bound, 'f', -1, 64), - }) - } - - { - // Generate series with "_bucket" suffix and "le" label. - addSample("_bucket", float64(count), sample, [2]string{ - "le", - "+Inf", - }) - } - } - return nil - }, - }); err != nil { - return nil, errors.Wrap(err, "do query") - } - - var ( - result = make([]storage.Series, 0, len(set)) - lb labels.ScratchBuilder - ) - for _, s := range set { - lb.Reset() - for key, value := range s.labels { - lb.Add(key, value) - } - lb.Sort() - s.series.labels = lb.Labels() - result = append(result, s.series) - } - - return result, nil -} - func (p *promQuerier) queryExpHistograms(ctx context.Context, query string) ([]storage.Series, error) { type seriesWithLabels struct { series *series[expHistData] @@ -643,127 +468,6 @@ func (p *promQuerier) queryExpHistograms(ctx context.Context, query string) ([]s return result, nil } -func (p *promQuerier) querySummaries(ctx context.Context, query string) ([]storage.Series, error) { - type seriesWithLabels struct { - series *series[pointData] - labels map[string]string - } - type summarySample struct { - timestamp int64 - rawAttributes, rawResource string - attributes, resource map[string]string - flags pmetric.DataPointFlags - } - - var ( - set = map[seriesKey]seriesWithLabels{} - addSample = func( - name string, - val float64, - sample summarySample, - bucketKey [2]string, - ) { - key := seriesKey{ - name: name, - attributes: sample.rawAttributes, - resource: sample.rawResource, - bucketKey: bucketKey, - } - s, ok := set[key] - if !ok { - s = seriesWithLabels{ - series: &series[pointData]{}, - labels: map[string]string{}, - } - set[key] = s - } - - if sample.flags.NoRecordedValue() { - val = math.Float64frombits(value.StaleNaN) - } - s.series.data.values = append(s.series.data.values, val) - s.series.ts = append(s.series.ts, sample.timestamp) - - s.labels["__name__"] = name - maps.Copy(s.labels, sample.attributes) - maps.Copy(s.labels, sample.resource) - if key := bucketKey[0]; key != "" { - s.labels[key] = bucketKey[1] - } - } - c = newSummaryColumns() - ) - if err := p.ch.Do(ctx, ch.Query{ - Body: query, - Result: c.Result(), - OnResult: func(ctx context.Context, block proto.Block) error { - for i := 0; i < c.timestamp.Rows(); i++ { - name := c.name.Row(i) - timestamp := c.timestamp.Row(i) - count := c.count.Row(i) - sum := c.sum.Row(i) - quantiles := c.quantiles.Row(i) - values := c.values.Row(i) - flags := pmetric.DataPointFlags(c.flags.Row(i)) - rawAttributes := c.attributes.Row(i) - rawResource := c.resource.Row(i) - - var ( - resource = map[string]string{} - attributes = map[string]string{} - ) - if err := parseLabels(rawResource, resource); err != nil { - return errors.Wrap(err, "parse resource") - } - if err := parseLabels(rawAttributes, attributes); err != nil { - return errors.Wrap(err, "parse attributes") - } - sample := summarySample{ - timestamp: timestamp.UnixMilli(), - rawAttributes: rawAttributes, - rawResource: rawResource, - attributes: attributes, - resource: resource, - flags: flags, - } - - addSample(name+"_count", float64(count), sample, [2]string{}) - addSample(name+"_sum", sum, sample, [2]string{}) - - for i := 0; i < min(len(quantiles), len(values)); i++ { - quantile := quantiles[i] - value := values[i] - - // Generate series with "quantile" label. - addSample("", value, sample, [2]string{ - "quantile", - strconv.FormatFloat(quantile, 'f', -1, 64), - }) - } - } - return nil - }, - }); err != nil { - return nil, errors.Wrap(err, "do query") - } - - var ( - result = make([]storage.Series, 0, len(set)) - lb labels.ScratchBuilder - ) - for _, s := range set { - lb.Reset() - for key, value := range s.labels { - lb.Add(key, value) - } - lb.Sort() - s.series.labels = lb.Labels() - result = append(result, s.series) - } - - return result, nil -} - type seriesSet[S storage.Series] struct { set []S n int diff --git a/internal/chstorage/schema_metrics.go b/internal/chstorage/schema_metrics.go index 5a7fb590..1827edd2 100644 --- a/internal/chstorage/schema_metrics.go +++ b/internal/chstorage/schema_metrics.go @@ -7,12 +7,27 @@ import ( "github.com/go-faster/jx" ) +type metricMapping int8 + +const ( + noMapping metricMapping = iota + histogramCount + histogramSum + histogramMin + histogramMax + histogramBucket + summaryCount + summarySum + summaryQuantile +) + const ( pointsSchema = `CREATE TABLE IF NOT EXISTS %s ( name LowCardinality(String), timestamp DateTime64(9), + mapping Enum8(` + metricMappingDDL + `), value Float64, flags UInt32, @@ -21,6 +36,17 @@ const ( ) ENGINE = MergeTree() ORDER BY timestamp;` + metricMappingDDL = ` + 'NO_MAPPING' = 0, + 'HISTOGRAM_COUNT' = 1, + 'HISTOGRAM_SUM' = 2, + 'HISTOGRAM_MIN' = 3, + 'HISTOGRAM_MAX' = 4, + 'HISTOGRAM_BUCKET' = 5, + 'SUMMARY_COUNT' = 6, + 'SUMMARY_SUM' = 7, + 'SUMMARY_QUANTILE' = 8 + ` histogramsSchema = `CREATE TABLE IF NOT EXISTS %s ( name LowCardinality(String), diff --git a/internal/chstorage/schema_traces.go b/internal/chstorage/schema_traces.go index 1f3fbeda..f1dc6f8b 100644 --- a/internal/chstorage/schema_traces.go +++ b/internal/chstorage/schema_traces.go @@ -52,11 +52,11 @@ PRIMARY KEY (trace_id, span_id);` valueTypeDDL = `'EMPTY' = 0,'STR' = 1,'INT' = 2,'DOUBLE' = 3,'BOOL' = 4,'MAP' = 5,'SLICE' = 6,'BYTES' = 7` ) -func encodeAttributes(attrs pcommon.Map) string { +func encodeAttributes(attrs pcommon.Map, additional ...[2]string) string { e := jx.GetEncoder() defer jx.PutEncoder(e) - encodeMap(e, attrs) + encodeMap(e, attrs, additional...) return e.String() } @@ -87,8 +87,8 @@ func encodeValue(e *jx.Encoder, v pcommon.Value) { } } -func encodeMap(e *jx.Encoder, m pcommon.Map) { - if otelstorage.Attrs(m).IsZero() { +func encodeMap(e *jx.Encoder, m pcommon.Map, additional ...[2]string) { + if otelstorage.Attrs(m).IsZero() && len(additional) == 0 { e.ObjEmpty() return } @@ -98,6 +98,10 @@ func encodeMap(e *jx.Encoder, m pcommon.Map) { encodeValue(e, v) return true }) + for _, pair := range additional { + e.FieldStart(pair[0]) + e.Str(pair[1]) + } e.ObjEnd() } From 1aa9b951f2dbeb849e9dc2ccc6d40ecc99474f20 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Thu, 30 Nov 2023 05:03:11 +0300 Subject: [PATCH 091/112] test(prome2e): query histograms and summaries --- integration/prome2e/common_test.go | 36 +++++++++++++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/integration/prome2e/common_test.go b/integration/prome2e/common_test.go index 8297df6d..1494f51e 100644 --- a/integration/prome2e/common_test.go +++ b/integration/prome2e/common_test.go @@ -106,7 +106,7 @@ func runTest( } }) t.Run("Series", func(t *testing.T) { - t.Run("ByName", func(t *testing.T) { + t.Run("PointByName", func(t *testing.T) { a := require.New(t) r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ @@ -123,6 +123,40 @@ func runTest( a.Equal("prometheus_http_requests_total", labels["__name__"]) } }) + t.Run("HistogramByName", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1600000000.0`), + End: promapi.NewOptPrometheusTimestamp(`1800000000.0`), + Match: []string{ + `prometheus_http_request_duration_seconds_count{}`, + }, + }) + a.NoError(err) + + a.NotEmpty(r.Data) + for _, labels := range r.Data { + a.Equal("prometheus_http_request_duration_seconds_count", labels["__name__"]) + } + }) + t.Run("SummaryByName", func(t *testing.T) { + a := require.New(t) + + r, err := c.GetSeries(ctx, promapi.GetSeriesParams{ + Start: promapi.NewOptPrometheusTimestamp(`1600000000.0`), + End: promapi.NewOptPrometheusTimestamp(`1800000000.0`), + Match: []string{ + `go_gc_duration_seconds{}`, + }, + }) + a.NoError(err) + + a.NotEmpty(r.Data) + for _, labels := range r.Data { + a.Equal("go_gc_duration_seconds", labels["__name__"]) + } + }) t.Run("OneMatcher", func(t *testing.T) { a := require.New(t) From 9aeed1dc67206e0ef2b44d492e1d32798627cc0b Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 15:13:01 +0300 Subject: [PATCH 092/112] test(lokie2e): rework --- integration/lokie2e/_golden/logs.yml | 5535 ++++++++++++++++++++ integration/lokie2e/_testdata/logs.jsonl | 21 - integration/lokie2e/common_test.go | 133 +- integration/lokie2e/generate_test.go | 145 + integration/lokie2e/loki_e2e.go | 28 +- internal/chstorage/querier_logs.go | 4 - internal/chstorage/schema_logs.go | 10 +- internal/logql/logqlengine/eval_streams.go | 2 +- internal/logql/logqlengine/json.go | 2 +- internal/logql/logqlengine/label_set.go | 14 +- internal/logql/logqlengine/otel_adapter.go | 38 + 11 files changed, 5808 insertions(+), 124 deletions(-) create mode 100644 integration/lokie2e/_golden/logs.yml delete mode 100644 integration/lokie2e/_testdata/logs.jsonl create mode 100644 integration/lokie2e/generate_test.go create mode 100644 internal/logql/logqlengine/otel_adapter.go diff --git a/integration/lokie2e/_golden/logs.yml b/integration/lokie2e/_golden/logs.yml new file mode 100644 index 00000000..9a624a70 --- /dev/null +++ b/integration/lokie2e/_golden/logs.yml @@ -0,0 +1,5535 @@ +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459200120000000" + spanId: "" + timeUnixNano: "1609459200120000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459200240000000" + spanId: "" + timeUnixNano: "1609459200240000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459200360000000" + spanId: "" + timeUnixNano: "1609459200360000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459200480000000" + spanId: "" + timeUnixNano: "1609459200480000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459200600000000" + spanId: "" + timeUnixNano: "1609459200600000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459200720000000" + spanId: "" + timeUnixNano: "1609459200720000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459200840000000" + spanId: "" + timeUnixNano: "1609459200840000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459200960000000" + spanId: "" + timeUnixNano: "1609459200960000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459201080000000" + spanId: "" + timeUnixNano: "1609459201080000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459201200000000" + spanId: "" + timeUnixNano: "1609459201200000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459201320000000" + spanId: "" + timeUnixNano: "1609459201320000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459201440000000" + spanId: "" + timeUnixNano: "1609459201440000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459201560000000" + spanId: "" + timeUnixNano: "1609459201560000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459201680000000" + spanId: "" + timeUnixNano: "1609459201680000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459201800000000" + spanId: "" + timeUnixNano: "1609459201800000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459201920000000" + spanId: "" + timeUnixNano: "1609459201920000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459202040000000" + spanId: "" + timeUnixNano: "1609459202040000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459202160000000" + spanId: "" + timeUnixNano: "1609459202160000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459202280000000" + spanId: "" + timeUnixNano: "1609459202280000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459202400000000" + spanId: "" + timeUnixNano: "1609459202400000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: GET + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.1 + body: + stringValue: GET /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459202520000000" + spanId: "" + timeUnixNano: "1609459202520000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459202640000000" + spanId: "" + timeUnixNano: "1609459202640000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459202760000000" + spanId: "" + timeUnixNano: "1609459202760000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459202880000000" + spanId: "" + timeUnixNano: "1609459202880000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203000000000" + spanId: "" + timeUnixNano: "1609459203000000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203120000000" + spanId: "" + timeUnixNano: "1609459203120000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203240000000" + spanId: "" + timeUnixNano: "1609459203240000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203360000000" + spanId: "" + timeUnixNano: "1609459203360000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203480000000" + spanId: "" + timeUnixNano: "1609459203480000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203600000000" + spanId: "" + timeUnixNano: "1609459203600000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203720000000" + spanId: "" + timeUnixNano: "1609459203720000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203840000000" + spanId: "" + timeUnixNano: "1609459203840000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459203960000000" + spanId: "" + timeUnixNano: "1609459203960000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459204080000000" + spanId: "" + timeUnixNano: "1609459204080000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459204200000000" + spanId: "" + timeUnixNano: "1609459204200000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459204320000000" + spanId: "" + timeUnixNano: "1609459204320000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459204440000000" + spanId: "" + timeUnixNano: "1609459204440000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459204560000000" + spanId: "" + timeUnixNano: "1609459204560000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459204680000000" + spanId: "" + timeUnixNano: "1609459204680000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459204800000000" + spanId: "" + timeUnixNano: "1609459204800000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: DELETE + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: DELETE /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459204920000000" + spanId: "" + timeUnixNano: "1609459204920000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459205040000000" + spanId: "" + timeUnixNano: "1609459205040000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459205160000000" + spanId: "" + timeUnixNano: "1609459205160000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459205280000000" + spanId: "" + timeUnixNano: "1609459205280000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459205400000000" + spanId: "" + timeUnixNano: "1609459205400000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459205520000000" + spanId: "" + timeUnixNano: "1609459205520000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459205640000000" + spanId: "" + timeUnixNano: "1609459205640000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459205760000000" + spanId: "" + timeUnixNano: "1609459205760000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459205880000000" + spanId: "" + timeUnixNano: "1609459205880000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206000000000" + spanId: "" + timeUnixNano: "1609459206000000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206120000000" + spanId: "" + timeUnixNano: "1609459206120000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206240000000" + spanId: "" + timeUnixNano: "1609459206240000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206360000000" + spanId: "" + timeUnixNano: "1609459206360000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206480000000" + spanId: "" + timeUnixNano: "1609459206480000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206600000000" + spanId: "" + timeUnixNano: "1609459206600000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206720000000" + spanId: "" + timeUnixNano: "1609459206720000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206840000000" + spanId: "" + timeUnixNano: "1609459206840000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459206960000000" + spanId: "" + timeUnixNano: "1609459206960000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459207080000000" + spanId: "" + timeUnixNano: "1609459207080000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459207200000000" + spanId: "" + timeUnixNano: "1609459207200000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459207320000000" + spanId: "" + timeUnixNano: "1609459207320000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: POST /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459207440000000" + spanId: "" + timeUnixNano: "1609459207440000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459207560000000" + spanId: "" + timeUnixNano: "1609459207560000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459207680000000" + spanId: "" + timeUnixNano: "1609459207680000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459207800000000" + spanId: "" + timeUnixNano: "1609459207800000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459207920000000" + spanId: "" + timeUnixNano: "1609459207920000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459208040000000" + spanId: "" + timeUnixNano: "1609459208040000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459208160000000" + spanId: "" + timeUnixNano: "1609459208160000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459208280000000" + spanId: "" + timeUnixNano: "1609459208280000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459208400000000" + spanId: "" + timeUnixNano: "1609459208400000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459208520000000" + spanId: "" + timeUnixNano: "1609459208520000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459208640000000" + spanId: "" + timeUnixNano: "1609459208640000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459208760000000" + spanId: "" + timeUnixNano: "1609459208760000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459208880000000" + spanId: "" + timeUnixNano: "1609459208880000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209000000000" + spanId: "" + timeUnixNano: "1609459209000000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209120000000" + spanId: "" + timeUnixNano: "1609459209120000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209240000000" + spanId: "" + timeUnixNano: "1609459209240000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209360000000" + spanId: "" + timeUnixNano: "1609459209360000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209480000000" + spanId: "" + timeUnixNano: "1609459209480000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209600000000" + spanId: "" + timeUnixNano: "1609459209600000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PATCH + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: PATCH /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209720000000" + spanId: "" + timeUnixNano: "1609459209720000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209840000000" + spanId: "" + timeUnixNano: "1609459209840000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459209960000000" + spanId: "" + timeUnixNano: "1609459209960000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459210080000000" + spanId: "" + timeUnixNano: "1609459210080000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459210200000000" + spanId: "" + timeUnixNano: "1609459210200000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459210320000000" + spanId: "" + timeUnixNano: "1609459210320000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459210440000000" + spanId: "" + timeUnixNano: "1609459210440000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459210560000000" + spanId: "" + timeUnixNano: "1609459210560000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459210680000000" + spanId: "" + timeUnixNano: "1609459210680000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459210800000000" + spanId: "" + timeUnixNano: "1609459210800000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459210920000000" + spanId: "" + timeUnixNano: "1609459210920000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459211040000000" + spanId: "" + timeUnixNano: "1609459211040000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459211160000000" + spanId: "" + timeUnixNano: "1609459211160000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459211280000000" + spanId: "" + timeUnixNano: "1609459211280000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459211400000000" + spanId: "" + timeUnixNano: "1609459211400000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459211520000000" + spanId: "" + timeUnixNano: "1609459211520000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459211640000000" + spanId: "" + timeUnixNano: "1609459211640000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459211760000000" + spanId: "" + timeUnixNano: "1609459211760000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459211880000000" + spanId: "" + timeUnixNano: "1609459211880000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/1.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212000000000" + spanId: "" + timeUnixNano: "1609459212000000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 236.7.233.166 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212120000000" + spanId: "" + timeUnixNano: "1609459212120000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "500" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 500 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212240000000" + spanId: "" + timeUnixNano: "1609459212240000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: HEAD + - key: http.status_code + value: + intValue: "500" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: HEAD /api/v1/series 500 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212360000000" + spanId: "" + timeUnixNano: "1609459212360000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212480000000" + spanId: "" + timeUnixNano: "1609459212480000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212600000000" + spanId: "" + timeUnixNano: "1609459212600000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212720000000" + spanId: "" + timeUnixNano: "1609459212720000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212840000000" + spanId: "" + timeUnixNano: "1609459212840000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459212960000000" + spanId: "" + timeUnixNano: "1609459212960000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459213080000000" + spanId: "" + timeUnixNano: "1609459213080000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459213200000000" + spanId: "" + timeUnixNano: "1609459213200000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459213320000000" + spanId: "" + timeUnixNano: "1609459213320000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459213440000000" + spanId: "" + timeUnixNano: "1609459213440000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459213560000000" + spanId: "" + timeUnixNano: "1609459213560000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459213680000000" + spanId: "" + timeUnixNano: "1609459213680000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459213800000000" + spanId: "" + timeUnixNano: "1609459213800000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459213920000000" + spanId: "" + timeUnixNano: "1609459213920000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459214040000000" + spanId: "" + timeUnixNano: "1609459214040000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459214160000000" + spanId: "" + timeUnixNano: "1609459214160000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459214280000000" + spanId: "" + timeUnixNano: "1609459214280000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459214400000000" + spanId: "" + timeUnixNano: "1609459214400000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459214520000000" + spanId: "" + timeUnixNano: "1609459214520000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459214640000000" + spanId: "" + timeUnixNano: "1609459214640000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version +--- +resourceLogs: +- resource: + attributes: + - key: service.name + value: + stringValue: testService + - key: service.version + value: + stringValue: testVersion + - key: service.namespace + value: + stringValue: testNamespace + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scopeLogs: + - logRecords: + - attributes: + - key: http.method + value: + stringValue: PUT + - key: http.status_code + value: + intValue: "200" + - key: client.address + value: + stringValue: 200.1.1.1 + - key: protocol + value: + stringValue: HTTP/2.0 + body: + stringValue: PUT /api/v1/series 200 250 - 0.000 ms + flags: 1 + observedTimeUnixNano: "1609459214760000000" + spanId: "" + timeUnixNano: "1609459214760000000" + traceId: "" + schemaUrl: https://opentelemetry.io/schemas/1.21.0 + scope: + attributes: + - key: oteldb.name + value: + stringValue: testDB + droppedAttributesCount: 1 + name: name + version: version diff --git a/integration/lokie2e/_testdata/logs.jsonl b/integration/lokie2e/_testdata/logs.jsonl deleted file mode 100644 index eca2021f..00000000 --- a/integration/lokie2e/_testdata/logs.jsonl +++ /dev/null @@ -1,21 +0,0 @@ -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770531506389037","observedTimeUnixNano":"1688770533485875260","body":{"stringValue":"{\"host\":\"182.232.235.224\", \"user-identifier\":\"murray3567\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"POST\", \"request\": \"/monetize\", \"protocol\":\"HTTP/2.0\", \"status\":503, \"bytes\":536, \"referer\": \"https://www.legacyeyeballs.io/streamline/mindshare/value-added/metrics\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532258117999","observedTimeUnixNano":"1688770533485879030","body":{"stringValue":"{\"host\":\"222.2.138.72\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"PUT\", \"request\": \"/strategize/e-services/24%2f7\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22430, \"referer\": \"https://www.globalapplications.info/scale/scalable/morph/collaborative\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532508421534","observedTimeUnixNano":"1688770533485879850","body":{"stringValue":"{\"host\":\"114.175.148.98\", \"user-identifier\":\"trantow3854\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PUT\", \"request\": \"/killer\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":9706, \"referer\": \"http://www.leadproactive.net/supply-chains\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506099582","observedTimeUnixNano":"1688770533485882410","body":{"stringValue":"{\"host\":\"224.56.79.52\", \"user-identifier\":\"armstrong8550\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"DELETE\", \"request\": \"/applications/incentivize/deliver/24%2f365\", \"protocol\":\"HTTP/1.0\", \"status\":501, \"bytes\":8709, \"referer\": \"http://www.directoptimize.biz/plug-and-play/leverage\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506357477","observedTimeUnixNano":"1688770533485884770","body":{"stringValue":"{\"host\":\"59.254.188.104\", \"user-identifier\":\"shanahan3762\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"PUT\", \"request\": \"/content/productize/expedite/viral\", \"protocol\":\"HTTP/1.0\", \"status\":204, \"bytes\":22262, \"referer\": \"https://www.internalone-to-one.io/productize\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506095582","observedTimeUnixNano":"1688770533485886960","body":{"stringValue":"{\"host\":\"108.166.198.19\", \"user-identifier\":\"rempel2418\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"PUT\", \"request\": \"/infrastructures/dynamic/interfaces\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":12418, \"referer\": \"http://www.directutilize.info/revolutionize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506270749","observedTimeUnixNano":"1688770533485888960","body":{"stringValue":"{\"host\":\"32.172.191.106\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"PATCH\", \"request\": \"/next-generation/efficient\", \"protocol\":\"HTTP/1.0\", \"status\":301, \"bytes\":19187, \"referer\": \"http://www.humanarchitectures.org/content/cross-media/brand/extensible\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"301"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506356337","observedTimeUnixNano":"1688770533485891020","body":{"stringValue":"{\"host\":\"245.62.143.62\", \"user-identifier\":\"kshlerin1534\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"GET\", \"request\": \"/architect/portals/niches\", \"protocol\":\"HTTP/1.0\", \"status\":403, \"bytes\":19870, \"referer\": \"http://www.directtransition.biz/mesh/next-generation/roi\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770532007261105","observedTimeUnixNano":"1688770533485893360","body":{"stringValue":"{\"host\":\"112.190.183.173\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"GET\", \"request\": \"/dynamic/sticky\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":21741, \"referer\": \"https://www.humaninfrastructures.biz/users/collaborative/proactive/proactive\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"201"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506066453","observedTimeUnixNano":"1688770533485897459","body":{"stringValue":"{\"host\":\"117.12.231.81\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"PUT\", \"request\": \"/models/empower/open-source\", \"protocol\":\"HTTP/1.1\", \"status\":302, \"bytes\":27722, \"referer\": \"https://www.dynamicmindshare.org/e-business/intuitive/unleash/markets\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506070283","observedTimeUnixNano":"1688770533485899839","body":{"stringValue":"{\"host\":\"236.7.233.166\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"HEAD\", \"request\": \"/enterprise\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":20357, \"referer\": \"https://www.investorseize.net/cultivate/synergies/revolutionize/extend\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506277879","observedTimeUnixNano":"1688770533485900339","body":{"stringValue":"{\"host\":\"55.119.244.237\", \"user-identifier\":\"buckridge5168\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"HEAD\", \"request\": \"/sexy/synthesize/architectures/front-end\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":24908, \"referer\": \"http://www.nationale-markets.net/bricks-and-clicks\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531756857070","observedTimeUnixNano":"1688770533485902439","body":{"stringValue":"{\"host\":\"14.182.107.209\", \"user-identifier\":\"kozey6276\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"DELETE\", \"request\": \"/embrace/cutting-edge/empower/empower\", \"protocol\":\"HTTP/1.0\", \"status\":405, \"bytes\":19380, \"referer\": \"http://www.legacycompelling.name/drive/strategic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506329078","observedTimeUnixNano":"1688770533485904529","body":{"stringValue":"{\"host\":\"80.210.177.228\", \"user-identifier\":\"nitzsche3256\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"GET\", \"request\": \"/paradigms/innovate/b2b/drive\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":923, \"referer\": \"http://www.legacyend-to-end.io/enhance\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506331908","observedTimeUnixNano":"1688770533485906569","body":{"stringValue":"{\"host\":\"86.173.91.15\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:30 +0000\", \"method\": \"PATCH\", \"request\": \"/open-source/integrate/open-source/visualize\", \"protocol\":\"HTTP/2.0\", \"status\":200, \"bytes\":19653, \"referer\": \"https://www.principalgenerate.io/visualize/dot-com\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506091172","observedTimeUnixNano":"1688770533485908829","body":{"stringValue":"{\"host\":\"214.52.159.245\", \"user-identifier\":\"lowe8277\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"POST\", \"request\": \"/transition/synthesize/applications/24%2f365\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":13965, \"referer\": \"https://www.leadnext-generation.info/redefine/e-business\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506274739","observedTimeUnixNano":"1688770533485910989","body":{"stringValue":"{\"host\":\"79.228.108.32\", \"user-identifier\":\"sanford2465\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"POST\", \"request\": \"/incentivize\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":9074, \"referer\": \"http://www.chiefe-business.info/e-enable/incentivize/transition\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506330508","observedTimeUnixNano":"1688770533485913019","body":{"stringValue":"{\"host\":\"153.126.163.17\", \"user-identifier\":\"terry2422\", \"datetime\":\"07/Jul/2023:22:55:29 +0000\", \"method\": \"DELETE\", \"request\": \"/e-enable\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":1033, \"referer\": \"http://www.customere-commerce.info/mesh\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506406776","observedTimeUnixNano":"1688770533485915089","body":{"stringValue":"{\"host\":\"106.153.80.249\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:31 +0000\", \"method\": \"POST\", \"request\": \"/systems\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":26505, \"referer\": \"http://www.chiefsolutions.info/technologies/b2c/embrace/applications\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506063113","observedTimeUnixNano":"1688770533485917269","body":{"stringValue":"{\"host\":\"139.211.170.136\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:26 +0000\", \"method\": \"GET\", \"request\": \"/iterate/implement/relationships\", \"protocol\":\"HTTP/1.1\", \"status\":100, \"bytes\":22426, \"referer\": \"https://www.legacyseamless.io/transparent\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506072323","observedTimeUnixNano":"1688770533485920149","body":{"stringValue":"{\"host\":\"159.238.229.29\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:27 +0000\", \"method\": \"POST\", \"request\": \"/world-class\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":22475, \"referer\": \"https://www.nationalend-to-end.io/cross-platform/visualize/compelling/virtual\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770531506267519","observedTimeUnixNano":"1688770533485920629","body":{"stringValue":"{\"host\":\"240.39.214.21\", \"user-identifier\":\"yost2184\", \"datetime\":\"07/Jul/2023:22:55:28 +0000\", \"method\": \"POST\", \"request\": \"/initiatives/benchmark/mesh/utilize\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":6318, \"referer\": \"http://www.regionalreintermediate.biz/innovative/utilize/morph\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770532759320597","observedTimeUnixNano":"1688770534505475760","body":{"stringValue":"{\"host\":\"219.253.131.24\", \"user-identifier\":\"nolan2452\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"POST\", \"request\": \"/transform/partnerships/compelling\", \"protocol\":\"HTTP/1.0\", \"status\":504, \"bytes\":5432, \"referer\": \"http://www.dynamicpartnerships.info/distributed/e-enable\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"504"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533009722650","observedTimeUnixNano":"1688770534505481330","body":{"stringValue":"{\"host\":\"140.150.10.211\", \"user-identifier\":\"stark5567\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PUT\", \"request\": \"/visionary/harness/exploit\", \"protocol\":\"HTTP/1.0\", \"status\":416, \"bytes\":12222, \"referer\": \"https://www.internalvortals.io/initiatives/niches\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533260610181","observedTimeUnixNano":"1688770534505485390","body":{"stringValue":"{\"host\":\"255.39.188.234\", \"user-identifier\":\"cartwright7210\", \"datetime\":\"07/Jul/2023:22:55:32 +0000\", \"method\": \"PATCH\", \"request\": \"/best-of-breed/transparent\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":28207, \"referer\": \"https://www.productvertical.com/models/harness/bandwidth/scalable\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533511255568","observedTimeUnixNano":"1688770534505488520","body":{"stringValue":"{\"host\":\"32.3.226.175\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"PUT\", \"request\": \"/communities/evolve/content/interfaces\", \"protocol\":\"HTTP/1.0\", \"status\":200, \"bytes\":16759, \"referer\": \"https://www.globalnext-generation.info/seize/engage/unleash/empower\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770533761830695","observedTimeUnixNano":"1688770534505491520","body":{"stringValue":"{\"host\":\"224.24.229.231\", \"user-identifier\":\"runte3687\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"POST\", \"request\": \"/e-business/brand/methodologies\", \"protocol\":\"HTTP/2.0\", \"status\":416, \"bytes\":13103, \"referer\": \"https://www.regionalrich.com/action-items\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534012191357","observedTimeUnixNano":"1688770534505494330","body":{"stringValue":"{\"host\":\"34.60.133.32\", \"user-identifier\":\"wisoky3066\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"PUT\", \"request\": \"/out-of-the-box\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":5665, \"referer\": \"http://www.leadrevolutionize.org/efficient/partnerships/architect/solutions\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534263072946","observedTimeUnixNano":"1688770534505497230","body":{"stringValue":"{\"host\":\"138.73.65.106\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:33 +0000\", \"method\": \"HEAD\", \"request\": \"/deploy/front-end/vertical/metrics\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":28841, \"referer\": \"https://www.regionalsyndicate.name/solutions\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770535014595986","observedTimeUnixNano":"1688770535605373502","body":{"stringValue":"{\"host\":\"119.217.207.214\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"POST\", \"request\": \"/magnetic/reinvent/virtual\", \"protocol\":\"HTTP/2.0\", \"status\":204, \"bytes\":23155, \"referer\": \"http://www.leadbandwidth.com/brand/unleash\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770535265453194","observedTimeUnixNano":"1688770535605378152","body":{"stringValue":"{\"host\":\"20.82.163.90\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"GET\", \"request\": \"/seamless/seamless/recontextualize\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":7620, \"referer\": \"http://www.leaduser-centric.io/reinvent/engineer/experiences\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770535515889000","observedTimeUnixNano":"1688770535605381162","body":{"stringValue":"{\"host\":\"75.110.63.232\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PUT\", \"request\": \"/one-to-one/aggregate\", \"protocol\":\"HTTP/1.1\", \"status\":416, \"bytes\":21091, \"referer\": \"https://www.regionalimplement.name/relationships/engage/e-services/redefine\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534514124542","observedTimeUnixNano":"1688770535605383892","body":{"stringValue":"{\"host\":\"150.204.109.167\", \"user-identifier\":\"swift5175\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"HEAD\", \"request\": \"/integrated/distributed/methodologies\", \"protocol\":\"HTTP/2.0\", \"status\":404, \"bytes\":10857, \"referer\": \"https://www.seniorsexy.com/distributed\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770534764273486","observedTimeUnixNano":"1688770535605386732","body":{"stringValue":"{\"host\":\"15.176.190.242\", \"user-identifier\":\"champlin5148\", \"datetime\":\"07/Jul/2023:22:55:34 +0000\", \"method\": \"PUT\", \"request\": \"/monetize/mindshare/rich/extensible\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":17396, \"referer\": \"http://www.globalrepurpose.com/redefine/brand/architect\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770535766768587","observedTimeUnixNano":"1688770536805705482","body":{"stringValue":"{\"host\":\"159.240.99.65\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"GET\", \"request\": \"/seamless\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":4573, \"referer\": \"https://www.forwardgenerate.org/revolutionize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536017199113","observedTimeUnixNano":"1688770536805710712","body":{"stringValue":"{\"host\":\"254.246.88.15\", \"user-identifier\":\"feest4001\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PUT\", \"request\": \"/architectures\", \"protocol\":\"HTTP/1.1\", \"status\":504, \"bytes\":9703, \"referer\": \"http://www.seniorgenerate.name/roi/web-enabled/mesh\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"504"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536267969591","observedTimeUnixNano":"1688770536805714342","body":{"stringValue":"{\"host\":\"117.113.156.145\", \"user-identifier\":\"hansen6831\", \"datetime\":\"07/Jul/2023:22:55:35 +0000\", \"method\": \"PATCH\", \"request\": \"/best-of-breed/strategize/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":500, \"bytes\":7590, \"referer\": \"http://www.internalnext-generation.biz/granular/relationships\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536518702679","observedTimeUnixNano":"1688770536805717532","body":{"stringValue":"{\"host\":\"108.138.59.166\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"GET\", \"request\": \"/proactive\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":20470, \"referer\": \"https://www.corporatevisionary.org/models/integrate/supply-chains/visualize\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"502"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770536769226151","observedTimeUnixNano":"1688770536805720572","body":{"stringValue":"{\"host\":\"80.21.65.147\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"POST\", \"request\": \"/methodologies\", \"protocol\":\"HTTP/1.1\", \"status\":504, \"bytes\":24809, \"referer\": \"https://www.leade-tailers.io/front-end/turn-key/supply-chains\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"504"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770537019590646","observedTimeUnixNano":"1688770538105581695","body":{"stringValue":"{\"host\":\"48.48.132.74\", \"user-identifier\":\"sauer4564\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"PATCH\", \"request\": \"/ubiquitous/transparent\", \"protocol\":\"HTTP/1.1\", \"status\":204, \"bytes\":21351, \"referer\": \"http://www.corporatebrand.com/synergies\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"204"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537270539598","observedTimeUnixNano":"1688770538105586155","body":{"stringValue":"{\"host\":\"41.84.80.177\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:36 +0000\", \"method\": \"POST\", \"request\": \"/schemas\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":9416, \"referer\": \"http://www.forwardrich.name/bleeding-edge\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537520765634","observedTimeUnixNano":"1688770538105589205","body":{"stringValue":"{\"host\":\"215.39.102.24\", \"user-identifier\":\"pouros8556\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"HEAD\", \"request\": \"/infrastructures\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":3738, \"referer\": \"https://www.internationalrevolutionary.io/incentivize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770537771653837","observedTimeUnixNano":"1688770538105592045","body":{"stringValue":"{\"host\":\"233.20.118.85\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"PATCH\", \"request\": \"/bandwidth/whiteboard/relationships/b2b\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":13768, \"referer\": \"https://www.internalchannels.io/synthesize/customized/solutions/target\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538022089288","observedTimeUnixNano":"1688770538105594684","body":{"stringValue":"{\"host\":\"76.247.19.239\", \"user-identifier\":\"cormier5083\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"DELETE\", \"request\": \"/interfaces\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":4639, \"referer\": \"https://www.principalengineer.biz/solutions/synergistic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770538272986089","observedTimeUnixNano":"1688770539304866280","body":{"stringValue":"{\"host\":\"182.27.181.198\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:37 +0000\", \"method\": \"HEAD\", \"request\": \"/deliver/e-tailers\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":13889, \"referer\": \"http://www.investorextensible.com/empower/clicks-and-mortar/dot-com\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538523374810","observedTimeUnixNano":"1688770539304871430","body":{"stringValue":"{\"host\":\"135.212.14.226\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/integrate/e-business/orchestrate\", \"protocol\":\"HTTP/1.1\", \"status\":416, \"bytes\":25334, \"referer\": \"http://www.investordynamic.name/benchmark/mission-critical\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"416"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770538774176503","observedTimeUnixNano":"1688770539304874460","body":{"stringValue":"{\"host\":\"47.20.233.211\", \"user-identifier\":\"schowalter7876\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/bricks-and-clicks/channels\", \"protocol\":\"HTTP/2.0\", \"status\":204, \"bytes\":11038, \"referer\": \"http://www.dynamicmission-critical.info/eyeballs\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539275422243","observedTimeUnixNano":"1688770539304875030","body":{"stringValue":"{\"host\":\"251.240.185.7\", \"user-identifier\":\"kirlin6527\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"HEAD\", \"request\": \"/embrace/next-generation/value-added\", \"protocol\":\"HTTP/1.0\", \"status\":204, \"bytes\":2768, \"referer\": \"http://www.leadembrace.io/brand\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"204"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539024541514","observedTimeUnixNano":"1688770539304877770","body":{"stringValue":"{\"host\":\"225.112.239.87\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:38 +0000\", \"method\": \"PATCH\", \"request\": \"/vertical/customized\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":25553, \"referer\": \"https://www.districtenterprise.net/schemas/drive\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"500"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770539525776242","observedTimeUnixNano":"1688770540604713526","body":{"stringValue":"{\"host\":\"110.213.172.140\", \"user-identifier\":\"flatley2072\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"POST\", \"request\": \"/back-end/plug-and-play\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":1079, \"referer\": \"https://www.corporatesyndicate.net/proactive/end-to-end/out-of-the-box/partnerships\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770539776638361","observedTimeUnixNano":"1688770540604718996","body":{"stringValue":"{\"host\":\"132.59.120.128\", \"user-identifier\":\"kautzer2610\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"GET\", \"request\": \"/embrace/mission-critical\", \"protocol\":\"HTTP/2.0\", \"status\":403, \"bytes\":9174, \"referer\": \"https://www.humanengineer.name/engineer/value-added/interactive\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540027065299","observedTimeUnixNano":"1688770540604723106","body":{"stringValue":"{\"host\":\"255.128.41.196\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"GET\", \"request\": \"/dynamic\", \"protocol\":\"HTTP/2.0\", \"status\":502, \"bytes\":23462, \"referer\": \"http://www.seniorenterprise.net/best-of-breed/grow/cross-media\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540277926986","observedTimeUnixNano":"1688770540604726566","body":{"stringValue":"{\"host\":\"108.113.41.211\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:39 +0000\", \"method\": \"HEAD\", \"request\": \"/cutting-edge/deliverables/cross-media\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":18274, \"referer\": \"https://www.directvisualize.io/vortals/24/7\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770540528287773","observedTimeUnixNano":"1688770540604729666","body":{"stringValue":"{\"host\":\"126.219.241.28\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"GET\", \"request\": \"/expedite/engineer\", \"protocol\":\"HTTP/2.0\", \"status\":503, \"bytes\":9425, \"referer\": \"http://www.internalrecontextualize.info/content/facilitate\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770540779141751","observedTimeUnixNano":"1688770541805060126","body":{"stringValue":"{\"host\":\"75.100.204.75\", \"user-identifier\":\"stiedemann6660\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"PATCH\", \"request\": \"/incubate/syndicate/communities/enhance\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":28778, \"referer\": \"https://www.principalmarkets.biz/web-readiness/holistic\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541029484568","observedTimeUnixNano":"1688770541805067016","body":{"stringValue":"{\"host\":\"164.61.170.176\", \"user-identifier\":\"feil3053\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"PUT\", \"request\": \"/deliver\", \"protocol\":\"HTTP/1.1\", \"status\":500, \"bytes\":12408, \"referer\": \"https://www.seniordot-com.net/best-of-breed/wireless/partnerships/magnetic\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"500"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541280380253","observedTimeUnixNano":"1688770541805070706","body":{"stringValue":"{\"host\":\"237.229.232.170\", \"user-identifier\":\"cummings3317\", \"datetime\":\"07/Jul/2023:22:55:40 +0000\", \"method\": \"HEAD\", \"request\": \"/technologies\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":23099, \"referer\": \"https://www.nationalsynergistic.net/cross-media/frictionless\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541531131440","observedTimeUnixNano":"1688770541805073826","body":{"stringValue":"{\"host\":\"191.42.120.4\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"DELETE\", \"request\": \"/cultivate/enterprise\", \"protocol\":\"HTTP/1.1\", \"status\":503, \"bytes\":1096, \"referer\": \"http://www.regionalend-to-end.com/e-commerce/bricks-and-clicks\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770541781600583","observedTimeUnixNano":"1688770541805077586","body":{"stringValue":"{\"host\":\"115.72.122.202\", \"user-identifier\":\"swift7514\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"PATCH\", \"request\": \"/transition\", \"protocol\":\"HTTP/2.0\", \"status\":405, \"bytes\":2870, \"referer\": \"https://www.leadenvisioneer.biz/compelling/killer\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770542031960818","observedTimeUnixNano":"1688770543105018384","body":{"stringValue":"{\"host\":\"125.249.12.227\", \"user-identifier\":\"crona1635\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"GET\", \"request\": \"/schemas\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":23913, \"referer\": \"https://www.leadinteractive.name/synthesize/deliverables/unleash/viral\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542784196884","observedTimeUnixNano":"1688770543105020304","body":{"stringValue":"{\"host\":\"31.184.47.132\", \"user-identifier\":\"boyer7325\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"GET\", \"request\": \"/24%2f7/web+services/extensible/reintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":17233, \"referer\": \"https://www.leadredefine.com/morph/exploit/seize/whiteboard\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542282846871","observedTimeUnixNano":"1688770543105023974","body":{"stringValue":"{\"host\":\"180.227.124.253\", \"user-identifier\":\"glover2525\", \"datetime\":\"07/Jul/2023:22:55:41 +0000\", \"method\": \"HEAD\", \"request\": \"/roi/revolutionary\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":4997, \"referer\": \"https://www.investorrecontextualize.net/cross-platform/syndicate/initiatives/content\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770542533294682","observedTimeUnixNano":"1688770543105026754","body":{"stringValue":"{\"host\":\"35.167.110.144\", \"user-identifier\":\"kiehn6344\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"PATCH\", \"request\": \"/user-centric/incubate/disintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":503, \"bytes\":19773, \"referer\": \"https://www.investorvertical.info/drive/streamline/e-services/reinvent\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543034516918","observedTimeUnixNano":"1688770543105029564","body":{"stringValue":"{\"host\":\"135.76.24.123\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"GET\", \"request\": \"/infrastructures\", \"protocol\":\"HTTP/1.1\", \"status\":501, \"bytes\":29365, \"referer\": \"http://www.dynamicweb-enabled.com/web-readiness/markets/revolutionize\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770543285385529","observedTimeUnixNano":"1688770544305313540","body":{"stringValue":"{\"host\":\"204.141.204.194\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:42 +0000\", \"method\": \"DELETE\", \"request\": \"/rich\", \"protocol\":\"HTTP/1.1\", \"status\":401, \"bytes\":6762, \"referer\": \"https://www.principaldeliver.biz/reintermediate/channels/impactful\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543535766279","observedTimeUnixNano":"1688770544305318969","body":{"stringValue":"{\"host\":\"25.50.100.83\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"DELETE\", \"request\": \"/sexy/web-readiness/infomediaries\", \"protocol\":\"HTTP/2.0\", \"status\":400, \"bytes\":8067, \"referer\": \"https://www.customermethodologies.com/user-centric/schemas/deliverables/synergize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770543786551452","observedTimeUnixNano":"1688770544305322659","body":{"stringValue":"{\"host\":\"172.52.222.78\", \"user-identifier\":\"waelchi8475\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"PUT\", \"request\": \"/leverage/syndicate\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":13351, \"referer\": \"http://www.dynamicweb-readiness.net/vertical/holistic/magnetic/content\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544037029250","observedTimeUnixNano":"1688770544305326179","body":{"stringValue":"{\"host\":\"18.33.245.88\", \"user-identifier\":\"brekke5341\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"HEAD\", \"request\": \"/world-class\", \"protocol\":\"HTTP/1.1\", \"status\":100, \"bytes\":23594, \"referer\": \"https://www.districtrevolutionary.org/brand\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544287848500","observedTimeUnixNano":"1688770544305331319","body":{"stringValue":"{\"host\":\"95.146.146.56\", \"user-identifier\":\"witting8863\", \"datetime\":\"07/Jul/2023:22:55:43 +0000\", \"method\": \"POST\", \"request\": \"/content\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22835, \"referer\": \"https://www.internationalb2c.org/recontextualize/recontextualize/grow/transform\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"502"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770545540651551","observedTimeUnixNano":"1688770545605162902","body":{"stringValue":"{\"host\":\"72.242.71.239\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"POST\", \"request\": \"/global\", \"protocol\":\"HTTP/1.1\", \"status\":404, \"bytes\":3026, \"referer\": \"http://www.nationalinfomediaries.net/solutions/roi/innovative\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544538215259","observedTimeUnixNano":"1688770545605168132","body":{"stringValue":"{\"host\":\"221.31.17.132\", \"user-identifier\":\"hammes8155\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"PUT\", \"request\": \"/innovate\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":18182, \"referer\": \"http://www.districtempower.name/architectures\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770544789083788","observedTimeUnixNano":"1688770545605171772","body":{"stringValue":"{\"host\":\"73.25.185.22\", \"user-identifier\":\"schaefer3552\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"HEAD\", \"request\": \"/collaborative/vertical\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":13514, \"referer\": \"https://www.legacyproductize.net/interfaces\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545039444047","observedTimeUnixNano":"1688770545605175002","body":{"stringValue":"{\"host\":\"133.159.101.254\", \"user-identifier\":\"bernier7784\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"POST\", \"request\": \"/deploy/synergize\", \"protocol\":\"HTTP/1.1\", \"status\":405, \"bytes\":4079, \"referer\": \"http://www.internalinterfaces.io/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545290299404","observedTimeUnixNano":"1688770545605178022","body":{"stringValue":"{\"host\":\"63.166.3.113\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:44 +0000\", \"method\": \"PUT\", \"request\": \"/networks/e-enable\", \"protocol\":\"HTTP/2.0\", \"status\":301, \"bytes\":23413, \"referer\": \"http://www.leadcontent.info/portals/content/transparent\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"301"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770546292750429","observedTimeUnixNano":"1688770546805390176","body":{"stringValue":"{\"host\":\"49.154.204.143\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"PATCH\", \"request\": \"/solutions\", \"protocol\":\"HTTP/2.0\", \"status\":405, \"bytes\":12772, \"referer\": \"https://www.chieftechnologies.com/deliver\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"405"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546543451957","observedTimeUnixNano":"1688770546805394616","body":{"stringValue":"{\"host\":\"139.113.227.72\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"PATCH\", \"request\": \"/users/dynamic/revolutionary\", \"protocol\":\"HTTP/1.1\", \"status\":203, \"bytes\":11927, \"referer\": \"http://www.investorsynergies.org/leading-edge/productize/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546793929399","observedTimeUnixNano":"1688770546805397786","body":{"stringValue":"{\"host\":\"18.225.144.149\", \"user-identifier\":\"macejkovic4811\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"DELETE\", \"request\": \"/cross-platform/optimize/e-tailers\", \"protocol\":\"HTTP/1.0\", \"status\":302, \"bytes\":8359, \"referer\": \"https://www.districtmesh.name/channels/partnerships/utilize/plug-and-play\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770545791539778","observedTimeUnixNano":"1688770546805401576","body":{"stringValue":"{\"host\":\"113.112.205.16\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"DELETE\", \"request\": \"/generate/schemas/impactful/frictionless\", \"protocol\":\"HTTP/1.0\", \"status\":501, \"bytes\":2572, \"referer\": \"https://www.chiefiterate.info/unleash\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"501"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770546041916043","observedTimeUnixNano":"1688770546805404055","body":{"stringValue":"{\"host\":\"147.251.113.164\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:45 +0000\", \"method\": \"GET\", \"request\": \"/integrated\", \"protocol\":\"HTTP/1.0\", \"status\":503, \"bytes\":10412, \"referer\": \"http://www.productrobust.io/strategic\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"503"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770547044430211","observedTimeUnixNano":"1688770548105337580","body":{"stringValue":"{\"host\":\"31.12.63.70\", \"user-identifier\":\"carroll3128\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"POST\", \"request\": \"/value-added/infomediaries\", \"protocol\":\"HTTP/1.1\", \"status\":201, \"bytes\":15247, \"referer\": \"https://www.principalopen-source.name/turn-key/facilitate/orchestrate\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547295243595","observedTimeUnixNano":"1688770548105342880","body":{"stringValue":"{\"host\":\"160.136.74.178\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:46 +0000\", \"method\": \"PUT\", \"request\": \"/productize/reintermediate/markets\", \"protocol\":\"HTTP/1.0\", \"status\":100, \"bytes\":28739, \"referer\": \"http://www.seniorplatforms.name/infrastructures\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"100"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547545668186","observedTimeUnixNano":"1688770548105346740","body":{"stringValue":"{\"host\":\"218.137.103.90\", \"user-identifier\":\"huels6041\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"POST\", \"request\": \"/enhance\", \"protocol\":\"HTTP/1.0\", \"status\":401, \"bytes\":1115, \"referer\": \"https://www.internalrevolutionary.com/roi/productize\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770547796500899","observedTimeUnixNano":"1688770548105350040","body":{"stringValue":"{\"host\":\"152.7.8.228\", \"user-identifier\":\"mosciski3102\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"PUT\", \"request\": \"/matrix/content\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":9350, \"referer\": \"http://www.humanreintermediate.com/architectures/web-enabled/24/365\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548046805313","observedTimeUnixNano":"1688770548105353230","body":{"stringValue":"{\"host\":\"189.145.183.88\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"HEAD\", \"request\": \"/mesh/b2b/syndicate/technologies\", \"protocol\":\"HTTP/1.1\", \"status\":200, \"bytes\":23769, \"referer\": \"http://www.humanparadigms.info/world-class\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"200"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770548548089763","observedTimeUnixNano":"1688770549305574358","body":{"stringValue":"{\"host\":\"149.219.8.124\", \"user-identifier\":\"johnston5032\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"DELETE\", \"request\": \"/convergence\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":20805, \"referer\": \"http://www.dynamicuser-centric.com/schemas/sexy/granular/bricks-and-clicks\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548798941415","observedTimeUnixNano":"1688770549305580998","body":{"stringValue":"{\"host\":\"202.159.240.173\", \"user-identifier\":\"hayes4575\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"POST\", \"request\": \"/cross-platform/content/functionalities/repurpose\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":21455, \"referer\": \"http://www.regionalgenerate.info/b2c/vortals/solutions\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549049313785","observedTimeUnixNano":"1688770549305588108","body":{"stringValue":"{\"host\":\"120.119.68.91\", \"user-identifier\":\"heidenreich7556\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"HEAD\", \"request\": \"/aggregate/networks\", \"protocol\":\"HTTP/1.0\", \"status\":205, \"bytes\":26555, \"referer\": \"http://www.dynamicvisionary.name/scalable\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549300083475","observedTimeUnixNano":"1688770549305592998","body":{"stringValue":"{\"host\":\"23.194.158.90\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:48 +0000\", \"method\": \"PATCH\", \"request\": \"/revolutionize/envisioneer/supply-chains\", \"protocol\":\"HTTP/1.1\", \"status\":302, \"bytes\":24515, \"referer\": \"https://www.dynamicproactive.io/customized/deliverables\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"302"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770548297678124","observedTimeUnixNano":"1688770549305597048","body":{"stringValue":"{\"host\":\"171.235.211.64\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:47 +0000\", \"method\": \"PUT\", \"request\": \"/proactive/expedite/seize/leading-edge\", \"protocol\":\"HTTP/1.1\", \"status\":403, \"bytes\":16631, \"referer\": \"https://www.humanengineer.info/extend\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770549550496684","observedTimeUnixNano":"1688770550605255542","body":{"stringValue":"{\"host\":\"35.243.14.54\", \"user-identifier\":\"weimann5622\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"DELETE\", \"request\": \"/seamless\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":27406, \"referer\": \"https://www.regionalembrace.biz/e-business/24/7\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550552786577","observedTimeUnixNano":"1688770550605257182","body":{"stringValue":"{\"host\":\"206.39.201.209\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"DELETE\", \"request\": \"/transform/infrastructures\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":17105, \"referer\": \"https://www.leadsticky.info/wireless/one-to-one\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770549801322883","observedTimeUnixNano":"1688770550605263581","body":{"stringValue":"{\"host\":\"101.240.9.159\", \"user-identifier\":\"turcotte5453\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"GET\", \"request\": \"/revolutionary/revolutionize/incubate/reintermediate\", \"protocol\":\"HTTP/1.1\", \"status\":406, \"bytes\":10229, \"referer\": \"http://www.forwardnetworks.org/generate/exploit/b2c/web services\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"406"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550051625492","observedTimeUnixNano":"1688770550605268311","body":{"stringValue":"{\"host\":\"148.16.104.181\", \"user-identifier\":\"bednar5574\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"GET\", \"request\": \"/collaborative/seize\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":13672, \"referer\": \"https://www.internationalinfrastructures.io/optimize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770550302524748","observedTimeUnixNano":"1688770550605271551","body":{"stringValue":"{\"host\":\"12.50.238.58\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:49 +0000\", \"method\": \"DELETE\", \"request\": \"/value-added/supply-chains\", \"protocol\":\"HTTP/1.0\", \"status\":200, \"bytes\":19948, \"referer\": \"http://www.investorefficient.biz/intuitive/24/7\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"200"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770550803736612","observedTimeUnixNano":"1688770551805598004","body":{"stringValue":"{\"host\":\"6.57.16.190\", \"user-identifier\":\"koelpin1807\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"HEAD\", \"request\": \"/mesh\", \"protocol\":\"HTTP/1.0\", \"status\":404, \"bytes\":10734, \"referer\": \"http://www.regionalbandwidth.com/integrate/benchmark\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"404"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551054058149","observedTimeUnixNano":"1688770551805607224","body":{"stringValue":"{\"host\":\"77.236.31.140\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"PATCH\", \"request\": \"/mission-critical\", \"protocol\":\"HTTP/2.0\", \"status\":201, \"bytes\":22561, \"referer\": \"https://www.directmaximize.com/supply-chains/deliverables/web-enabled\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551305051212","observedTimeUnixNano":"1688770551805613764","body":{"stringValue":"{\"host\":\"96.170.101.152\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:50 +0000\", \"method\": \"GET\", \"request\": \"/implement/cultivate/revolutionary\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":19505, \"referer\": \"http://www.directrobust.org/dot-com/benchmark\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770551555515704","observedTimeUnixNano":"1688770551805620524","body":{"stringValue":"{\"host\":\"250.164.115.76\", \"user-identifier\":\"hermiston4072\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PUT\", \"request\": \"/experiences\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":18299, \"referer\": \"http://www.directrobust.name/initiatives\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770551806068175","observedTimeUnixNano":"1688770552904845153","body":{"stringValue":"{\"host\":\"254.168.97.108\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PUT\", \"request\": \"/vertical/robust/incubate\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":12490, \"referer\": \"http://www.productweb services.name/redefine/engage/bleeding-edge/maximize\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PUT"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552056479008","observedTimeUnixNano":"1688770552904851803","body":{"stringValue":"{\"host\":\"118.204.193.76\", \"user-identifier\":\"kihn1514\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"DELETE\", \"request\": \"/dot-com/scalable/interfaces\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":1951, \"referer\": \"https://www.dynamicleading-edge.org/customized/deliverables\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"502"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552808876801","observedTimeUnixNano":"1688770552904853303","body":{"stringValue":"{\"host\":\"239.84.230.236\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"DELETE\", \"request\": \"/back-end/target\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":13592, \"referer\": \"http://www.corporatefacilitate.io/orchestrate/global\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552306744053","observedTimeUnixNano":"1688770552904857773","body":{"stringValue":"{\"host\":\"11.118.181.255\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:51 +0000\", \"method\": \"PATCH\", \"request\": \"/one-to-one/channels/networks/collaborative\", \"protocol\":\"HTTP/1.1\", \"status\":501, \"bytes\":25565, \"referer\": \"http://www.dynamicinterfaces.biz/deliverables/real-time/global\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770552557804932","observedTimeUnixNano":"1688770552904866042","body":{"stringValue":"{\"host\":\"38.157.64.60\", \"user-identifier\":\"collins3161\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"POST\", \"request\": \"/leverage/convergence\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":7451, \"referer\": \"http://www.dynamicapplications.name/scalable\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770553059904090","observedTimeUnixNano":"1688770554104914869","body":{"stringValue":"{\"host\":\"91.210.58.64\", \"user-identifier\":\"hirthe5885\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"POST\", \"request\": \"/experiences\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":26372, \"referer\": \"https://www.globalreintermediate.biz/scalable/strategic/deliver\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553310125793","observedTimeUnixNano":"1688770554104921949","body":{"stringValue":"{\"host\":\"171.146.184.207\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:52 +0000\", \"method\": \"HEAD\", \"request\": \"/global/wireless/initiatives\", \"protocol\":\"HTTP/2.0\", \"status\":203, \"bytes\":3559, \"referer\": \"http://www.districtextensible.info/sexy\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"203"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553561204179","observedTimeUnixNano":"1688770554104926178","body":{"stringValue":"{\"host\":\"223.135.210.193\", \"user-identifier\":\"lakin5175\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"DELETE\", \"request\": \"/e-commerce\", \"protocol\":\"HTTP/1.1\", \"status\":301, \"bytes\":17589, \"referer\": \"http://www.investortransform.org/embrace\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"301"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770553811251136","observedTimeUnixNano":"1688770554104929478","body":{"stringValue":"{\"host\":\"167.99.122.25\", \"user-identifier\":\"dietrich4410\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"DELETE\", \"request\": \"/visualize/impactful/bleeding-edge/portals\", \"protocol\":\"HTTP/2.0\", \"status\":501, \"bytes\":21701, \"referer\": \"http://www.seniorredefine.info/e-commerce\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"501"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554061484609","observedTimeUnixNano":"1688770554104932588","body":{"stringValue":"{\"host\":\"12.8.67.46\", \"user-identifier\":\"stracke8218\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"HEAD\", \"request\": \"/ubiquitous/morph/visualize\", \"protocol\":\"HTTP/1.0\", \"status\":401, \"bytes\":3223, \"referer\": \"http://www.regionaldisintermediate.name/communities/synergies/drive\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770555062845794","observedTimeUnixNano":"1688770555404999753","body":{"stringValue":"{\"host\":\"68.65.52.180\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"PATCH\", \"request\": \"/engage/web-enabled/virtual/metrics\", \"protocol\":\"HTTP/2.0\", \"status\":100, \"bytes\":22935, \"referer\": \"http://www.dynamicfront-end.info/revolutionize/architect\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"100"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554311549923","observedTimeUnixNano":"1688770555405006503","body":{"stringValue":"{\"host\":\"58.176.246.52\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:53 +0000\", \"method\": \"HEAD\", \"request\": \"/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":26702, \"referer\": \"http://www.corporateniches.biz/exploit\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"400"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770555313060774","observedTimeUnixNano":"1688770555405007793","body":{"stringValue":"{\"host\":\"219.158.106.54\", \"user-identifier\":\"treutel4445\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"HEAD\", \"request\": \"/granular/killer/enhance/leverage\", \"protocol\":\"HTTP/1.0\", \"status\":400, \"bytes\":7060, \"referer\": \"http://www.productmonetize.org/visualize/paradigms/orchestrate/seize\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"HEAD"}},{"key":"http.status","value":{"stringValue":"400"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554561762855","observedTimeUnixNano":"1688770555405012603","body":{"stringValue":"{\"host\":\"89.109.67.122\", \"user-identifier\":\"lang1745\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"PATCH\", \"request\": \"/syndicate\", \"protocol\":\"HTTP/1.0\", \"status\":304, \"bytes\":4309, \"referer\": \"http://www.chiefintuitive.biz/sexy/generate/end-to-end\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"304"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770554812778300","observedTimeUnixNano":"1688770555405016873","body":{"stringValue":"{\"host\":\"108.149.26.184\", \"user-identifier\":\"wyman8181\", \"datetime\":\"07/Jul/2023:22:55:54 +0000\", \"method\": \"DELETE\", \"request\": \"/infrastructures/utilize/mindshare\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":8395, \"referer\": \"https://www.dynamicintuitive.biz/efficient/eyeballs/seamless/enable\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770555564076188","observedTimeUnixNano":"1688770556605475977","body":{"stringValue":"{\"host\":\"214.112.255.154\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"POST\", \"request\": \"/metrics/24%2f7/deliverables/technologies\", \"protocol\":\"HTTP/1.1\", \"status\":205, \"bytes\":8201, \"referer\": \"http://www.centralfacilitate.net/convergence/repurpose\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"POST"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770555814316806","observedTimeUnixNano":"1688770556605481157","body":{"stringValue":"{\"host\":\"222.210.64.103\", \"user-identifier\":\"jaskolski1751\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"HEAD\", \"request\": \"/synergistic\", \"protocol\":\"HTTP/2.0\", \"status\":500, \"bytes\":1213, \"referer\": \"http://www.internationalbenchmark.info/cross-platform/infomediaries\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"HEAD"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556065384879","observedTimeUnixNano":"1688770556605484527","body":{"stringValue":"{\"host\":\"86.90.114.24\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"GET\", \"request\": \"/synergize/incubate/platforms\", \"protocol\":\"HTTP/1.0\", \"status\":201, \"bytes\":8681, \"referer\": \"http://www.forwardvisualize.info/technologies/holistic\"}"},"attributes":[{"key":"http.status","value":{"stringValue":"201"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556315409620","observedTimeUnixNano":"1688770556605487627","body":{"stringValue":"{\"host\":\"234.206.18.83\", \"user-identifier\":\"kozey6774\", \"datetime\":\"07/Jul/2023:22:55:55 +0000\", \"method\": \"GET\", \"request\": \"/synthesize/users/redefine\", \"protocol\":\"HTTP/1.0\", \"status\":500, \"bytes\":24463, \"referer\": \"http://www.legacyplug-and-play.net/mesh\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"500"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556566005240","observedTimeUnixNano":"1688770556605490717","body":{"stringValue":"{\"host\":\"144.131.190.47\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"PATCH\", \"request\": \"/roi\", \"protocol\":\"HTTP/1.1\", \"status\":502, \"bytes\":22695, \"referer\": \"http://www.centralsynergize.biz/proactive/next-generation\"}"},"attributes":[{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"502"}}],"traceId":"","spanId":""}]}]}]} -{"resourceLogs":[{"resource":{},"scopeLogs":[{"scope":{},"logRecords":[{"timeUnixNano":"1688770557066772308","observedTimeUnixNano":"1688770557905095435","body":{"stringValue":"{\"host\":\"182.117.211.164\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"PATCH\", \"request\": \"/content/viral\", \"protocol\":\"HTTP/1.0\", \"status\":403, \"bytes\":4228, \"referer\": \"http://www.internalproactive.io/benchmark/extend/roi/expedite\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"PATCH"}},{"key":"http.status","value":{"stringValue":"403"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557316918605","observedTimeUnixNano":"1688770557905100755","body":{"stringValue":"{\"host\":\"20.32.13.13\", \"user-identifier\":\"yost7451\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"DELETE\", \"request\": \"/transparent\", \"protocol\":\"HTTP/1.0\", \"status\":502, \"bytes\":20998, \"referer\": \"https://www.productarchitect.com/generate/paradigms\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"502"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557566998293","observedTimeUnixNano":"1688770557905104465","body":{"stringValue":"{\"host\":\"47.67.12.210\", \"user-identifier\":\"-\", \"datetime\":\"07/Jul/2023:22:55:57 +0000\", \"method\": \"GET\", \"request\": \"/vertical/architectures\", \"protocol\":\"HTTP/2.0\", \"status\":205, \"bytes\":4031, \"referer\": \"https://www.internalcollaborative.com/sexy/enhance/clicks-and-mortar\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"205"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770557817067841","observedTimeUnixNano":"1688770557905107475","body":{"stringValue":"{\"host\":\"178.160.245.169\", \"user-identifier\":\"stracke6888\", \"datetime\":\"07/Jul/2023:22:55:57 +0000\", \"method\": \"GET\", \"request\": \"/dynamic/24%2f7\", \"protocol\":\"HTTP/2.0\", \"status\":401, \"bytes\":24926, \"referer\": \"http://www.legacye-services.io/e-business/grow/seamless/seize\"}"},"attributes":[{"key":"http.method","value":{"stringValue":"GET"}},{"key":"http.status","value":{"stringValue":"401"}},{"key":"job","value":{"stringValue":"generated-logs"}},{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}}],"traceId":"","spanId":""},{"timeUnixNano":"1688770556816580510","observedTimeUnixNano":"1688770557905110645","body":{"stringValue":"{\"host\":\"244.189.75.59\", \"user-identifier\":\"jaskolski1628\", \"datetime\":\"07/Jul/2023:22:55:56 +0000\", \"method\": \"DELETE\", \"request\": \"/revolutionize/whiteboard/e-markets\", \"protocol\":\"HTTP/1.0\", \"status\":406, \"bytes\":29374, \"referer\": \"http://www.nationalfacilitate.com/real-time\"}"},"attributes":[{"key":"filename","value":{"stringValue":"/var/log/generated.txt"}},{"key":"http.method","value":{"stringValue":"DELETE"}},{"key":"http.status","value":{"stringValue":"406"}},{"key":"job","value":{"stringValue":"generated-logs"}}],"traceId":"","spanId":""}]}]}]} diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index 2ed51bcb..871e8129 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -1,6 +1,7 @@ package lokie2e_test import ( + "bytes" "context" "fmt" "net/http/httptest" @@ -8,9 +9,12 @@ import ( "testing" "time" + "github.com/go-faster/sdk/gold" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/plog" + "sigs.k8s.io/yaml" "github.com/go-faster/oteldb/integration/lokie2e" "github.com/go-faster/oteldb/internal/logql" @@ -21,15 +25,11 @@ import ( "github.com/go-faster/oteldb/internal/otelstorage" ) -func readBatchSet(p string) (s lokie2e.BatchSet, _ error) { - f, err := os.Open(p) - if err != nil { - return s, err - } - defer func() { - _ = f.Close() - }() - return lokie2e.ParseBatchSet(f) +func TestMain(m *testing.M) { + // Explicitly registering flags for golden files. + gold.Init() + + os.Exit(m.Run()) } func setupDB( @@ -41,12 +41,23 @@ func setupDB( engineQuerier logqlengine.Querier, ) *lokiapi.Client { consumer := logstorage.NewConsumer(inserter) + + logEncoder := plog.JSONMarshaler{} + var out bytes.Buffer for i, b := range set.Batches { if err := consumer.ConsumeLogs(ctx, b); err != nil { t.Fatalf("Send batch %d: %+v", i, err) } + data, err := logEncoder.MarshalLogs(b) + require.NoError(t, err) + outData, err := yaml.JSONToYAML(data) + require.NoError(t, err) + out.WriteString("---\n") + out.Write(outData) } + gold.Str(t, out.String(), "logs.yml") + engine := logqlengine.NewEngine(engineQuerier, logqlengine.Options{ ParseOptions: logql.ParseOptions{AllowDots: true}, }) @@ -70,7 +81,9 @@ func runTest( querier logstorage.Querier, engineQuerier logqlengine.Querier, ) { - set, err := readBatchSet("_testdata/logs.jsonl") + now := time.Date(2021, 1, 1, 0, 0, 0, 0, time.UTC) + set, err := generateLogs(now) + require.NoError(t, err) require.NoError(t, err) require.NotEmpty(t, set.Batches) require.NotEmpty(t, set.Labels) @@ -130,73 +143,75 @@ func runTest( }{ // Label matchers. // Effectively match GET. - {`{http.method="GET"}`, 21}, - {`{http.method=~".*GET.*"}`, 21}, - {`{http.method=~"^GET$"}`, 21}, - {`{http.method!~"(HEAD|POST|DELETE|PUT|PATCH|TRACE|OPTIONS)"}`, 21}, + {`{http_method="GET"}`, 21}, + {`{http_method=~".*GET.*"}`, 21}, + {`{http_method=~"^GET$"}`, 21}, + {`{http_method!~"(HEAD|POST|DELETE|PUT|PATCH|TRACE|OPTIONS)"}`, 21}, // Try other methods. - {`{http.method="DELETE"}`, 20}, - {`{http.method="GET"}`, 21}, - {`{http.method="HEAD"}`, 22}, - {`{http.method="PATCH"}`, 19}, - {`{http.method="POST"}`, 21}, - {`{http.method="PUT"}`, 20}, - {`{http.method="GET"} | json`, 21}, + {`{http_method="DELETE"}`, 20}, + {`{http_method="GET"}`, 21}, + {`{http_method="HEAD"}`, 22}, + {`{http_method="PATCH"}`, 19}, + {`{http_method="POST"}`, 21}, + {`{http_method="PUT"}`, 20}, + {`{http_method="GET"} | json`, 21}, // Negative label matcher. - {`{http.method!="HEAD"}`, len(set.Records) - 22}, - {`{http.method!~"^HEAD$"}`, len(set.Records) - 22}, + {`{http_method!="HEAD"}`, len(set.Records) - 22}, + {`{http_method!~"^HEAD$"}`, len(set.Records) - 22}, // Multiple lables. - {`{http.method="HEAD",http.status="500"}`, 2}, - {`{http.method="HEAD",http.status=~"^500$"}`, 2}, - {`{http.method=~".*HEAD.*",http.status=~"^500$"}`, 2}, + {`{http_method="HEAD",http_status_code="500"}`, 2}, + {`{http_method="HEAD",http_status_code=~"^500$"}`, 2}, + {`{http_method=~".*HEAD.*",http_status_code=~"^500$"}`, 2}, // Line filter. - {`{http.method=~".+"} |= "\"method\": \"GET\""`, 21}, - {`{http.method=~".+"} |= "\"method\": \"DELETE\""`, 20}, - {`{http.method=~".+"} |= "\"method\": \"HEAD\"" |= "\"status\":500"`, 2}, - {`{http.method=~".+"} |~ "\"method\":\\s*\"DELETE\""`, 20}, - {`{http.method=~".+"} |~ "\"method\":\\s*\"HEAD\"" |= "\"status\":500"`, 2}, + {`{http_method=~".+"} |= "GET"`, 21}, + {`{http_method=~".+"} |= "DELETE"`, 20}, + {`{http_method=~".+"} |= "HEAD" |= "500"`, 2}, + {`{http_method=~".+"} |~ "DELETE"`, 20}, + {`{http_method=~".+"} |~ "HEAD" |= "500"`, 2}, // Try to not use offloading. - {`{http.method=~".+"} | line_format "{{ __line__ }}" |= "\"method\": \"DELETE\""`, 20}, - {`{http.method=~".+"} | line_format "{{ __line__ }}" |= "\"method\": \"HEAD\"" |= "\"status\":500"`, 2}, - {`{http.method=~".+"} |= "\"method\": \"HEAD\"" | line_format "{{ __line__ }}" |= "\"status\":500"`, 2}, + {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "DELETE"`, 20}, + {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "HEAD" |= "500"`, 2}, + {`{http_method=~".+"} |= "HEAD" | line_format "{{ __line__ }}" |= "500"`, 2}, // Negative line matcher. - {`{http.method=~".+"} != "\"method\": \"HEAD\""`, len(set.Records) - 22}, - {`{http.method=~".+"} !~ "\"method\":\\s*\"HEAD\""`, len(set.Records) - 22}, + {`{http_method=~".+"} != "HEAD"`, len(set.Records) - 22}, + {`{http_method=~".+"} !~ "HEAD"`, len(set.Records) - 22}, // IP line filter. - {`{http.method="HEAD"} |= ip("236.7.233.166")`, 1}, + {`{http_method="HEAD"} |= ip("236.7.233.166")`, 1}, // Label filter. - {`{http.method=~".+"} | http.method = "GET"`, 21}, - {`{http.method=~".+"} | http.method = "HEAD", http.status = "500"`, 2}, + {`{http_method=~".+"} | http_method = "GET"`, 21}, + {`{http_method=~".+"} | http_method = "HEAD", http_status_code = "500"`, 2}, // Number of lines per protocol. // // "HTTP/1.0" 55 - // "HTTP/1.1" 38 - // "HTTP/2.0" 30 + // "HTTP/1.1" 10 + // "HTTP/2.0" 58 // - {`{http.method=~".+"} | json | protocol = "HTTP/1.0"`, 55}, - {`{http.method=~".+"} | json | protocol =~ "HTTP/1.\\d"`, 55 + 38}, - {`{http.method=~".+"} | json | protocol != "HTTP/2.0"`, 55 + 38}, - {`{http.method=~".+"} | json | protocol !~ "HTTP/2.\\d"`, 55 + 38}, + {`{http_method=~".+"} | json | protocol = "HTTP/1.0"`, 55}, + {`{http_method=~".+"} | json | protocol = "HTTP/1.1"`, 10}, + {`{http_method=~".+"} | json | protocol = "HTTP/2.0"`, 58}, + {`{http_method=~".+"} | json | protocol =~ "HTTP/1.\\d"`, 55 + 10}, + {`{http_method=~".+"} | json | protocol != "HTTP/2.0"`, 55 + 10}, + {`{http_method=~".+"} | json | protocol !~ "HTTP/2.\\d"`, 55 + 10}, // IP label filter. - {`{http.method="HEAD"} | json | host = "236.7.233.166"`, 1}, - {`{http.method="HEAD"} | json | host == ip("236.7.233.166")`, 1}, - {`{http.method="HEAD"} | json | host == ip("236.7.233.0/24")`, 1}, - {`{http.method="HEAD"} | json | host == ip("236.7.233.0-236.7.233.255")`, 1}, + {`{http_method="HEAD"} | client_address = "236.7.233.166"`, 1}, + {`{http_method="HEAD"} | client_address == ip("236.7.233.166")`, 1}, + {`{http_method="HEAD"} | client_address == ip("236.7.233.0/24")`, 1}, + {`{http_method="HEAD"} | client_address == ip("236.7.233.0-236.7.233.255")`, 1}, // Distinct filter. - {`{http.method=~".+"} | distinct http.method`, 6}, - {`{http.method=~".+"} | json | distinct method`, 6}, - {`{http.method=~".+"} | json | distinct protocol`, 3}, + {`{http_method=~".+"} | distinct http_method`, 6}, + {`{http_method=~".+"} | distinct protocol`, 3}, // Sure empty queries. - {`{http.method="GET"} | json | http.method != "GET"`, 0}, - {`{http.method="HEAD"} | clearly_not_exist > 0`, 0}, + {`{http_method="GET"} | http_method != "GET"`, 0}, + {`{http_method="HEAD"} | clearly_not_exist > 0`, 0}, } labelSetHasAttrs := func(t assert.TestingT, set lokiapi.LabelSet, attrs pcommon.Map) { // Do not check length, since label set may contain some parsed labels. attrs.Range(func(k string, v pcommon.Value) bool { + k = otelstorage.KeyToLabel(k) assert.Contains(t, set, k) assert.Equal(t, v.AsString(), set[k]) return true @@ -233,7 +248,9 @@ func runTest( record, ok := set.Records[pcommon.Timestamp(entry.T)] require.Truef(t, ok, "can't find log record %d", entry.T) - line := record.Body().AsString() + line := logqlengine.LineFromRecord( + logstorage.NewRecordFromOTEL(pcommon.NewResource(), pcommon.NewInstrumentationScope(), record), + ) assert.Equal(t, line, entry.V) labelSetHasAttrs(t, stream.Stream.Value, record.Attributes()) @@ -245,7 +262,7 @@ func runTest( }) t.Run("MetricQueries", func(t *testing.T) { resp, err := c.QueryRange(ctx, lokiapi.QueryRangeParams{ - Query: `sum by (http.method) ( count_over_time({http.method=~".+"} [30s]) )`, + Query: `sum by (http_method) ( count_over_time({http_method=~".+"} [30s]) )`, // Query all data in a one step. Start: lokiapi.NewOptLokiTime(asLokiTime(set.End)), End: lokiapi.NewOptLokiTime(asLokiTime(set.End + otelstorage.Timestamp(10*time.Second))), @@ -262,9 +279,9 @@ func runTest( methods := map[string]string{} for _, series := range matrix { labels := series.Metric.Value - assert.Contains(t, labels, "http.method") + assert.Contains(t, labels, "http_method") assert.Len(t, labels, 1) - method := labels["http.method"] + method := labels["http_method"] values := series.Values assert.Len(t, values, 1) diff --git a/integration/lokie2e/generate_test.go b/integration/lokie2e/generate_test.go new file mode 100644 index 00000000..b414ff6f --- /dev/null +++ b/integration/lokie2e/generate_test.go @@ -0,0 +1,145 @@ +package lokie2e_test + +import ( + "bytes" + "fmt" + "testing" + "time" + + "github.com/go-faster/sdk/gold" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/plog" + "go.opentelemetry.io/otel/attribute" + semconv "go.opentelemetry.io/otel/semconv/v1.21.0" + "sigs.k8s.io/yaml" + + "github.com/go-faster/oteldb/integration/lokie2e" +) + +func appendAttributes(target pcommon.Map, attrs []attribute.KeyValue) { + for _, attr := range attrs { + k := string(attr.Key) + switch attr.Value.Type() { + case attribute.BOOL: + target.PutBool(k, attr.Value.AsBool()) + case attribute.INT64: + target.PutInt(k, attr.Value.AsInt64()) + case attribute.FLOAT64: + target.PutDouble(k, attr.Value.AsFloat64()) + case attribute.STRING: + target.PutStr(k, attr.Value.AsString()) + } + } +} + +type httpLog struct { + Time time.Time + Method string + Status int + Bytes int + Protocol string + User string + URL string + IP string + Ref string +} + +func (l httpLog) Append(s *lokie2e.BatchSet) error { + var ( + ld = plog.NewLogs() + rl = ld.ResourceLogs().AppendEmpty() + ) + appendAttributes(rl.Resource().Attributes(), []attribute.KeyValue{ + semconv.ServiceName("testService"), + semconv.ServiceVersion("testVersion"), + semconv.ServiceNamespace("testNamespace"), + }) + rl.SetSchemaUrl(semconv.SchemaURL) + il := rl.ScopeLogs().AppendEmpty() + { + sc := il.Scope() + sc.SetName("name") + sc.SetVersion("version") + sc.Attributes().PutStr("oteldb.name", "testDB") + sc.SetDroppedAttributesCount(1) + } + il.SetSchemaUrl(semconv.SchemaURL) + lg := il.LogRecords().AppendEmpty() + lg.Body().SetStr(fmt.Sprintf("%s %s %d %d - 0.000 ms", l.Method, l.URL, l.Status, l.Bytes)) + lg.SetTimestamp(pcommon.NewTimestampFromTime(l.Time)) + lg.SetObservedTimestamp(pcommon.NewTimestampFromTime(l.Time)) + appendAttributes(lg.Attributes(), []attribute.KeyValue{ + semconv.HTTPMethod(l.Method), + semconv.HTTPStatusCode(l.Status), + semconv.ClientAddress(l.IP), + attribute.String("protocol", l.Protocol), + }) + lg.SetFlags(plog.DefaultLogRecordFlags.WithIsSampled(true)) + if err := s.Append(ld); err != nil { + return err + } + return nil +} + +func generateLogs(now time.Time) (s lokie2e.BatchSet, err error) { + type httpLogBatch struct { + Method string + Status int + Count int + IP string + Protocol string + } + var lines []httpLog + for _, b := range []httpLogBatch{ + {Method: "GET", Status: 200, Count: 11, IP: "200.1.1.1", Protocol: "HTTP/1.0"}, + {Method: "GET", Status: 200, Count: 10, IP: "200.1.1.1", Protocol: "HTTP/1.1"}, + {Method: "DELETE", Status: 200, Count: 20, IP: "200.1.1.1", Protocol: "HTTP/2.0"}, + {Method: "POST", Status: 200, Count: 21, IP: "200.1.1.1", Protocol: "HTTP/1.0"}, + {Method: "PATCH", Status: 200, Count: 19, IP: "200.1.1.1", Protocol: "HTTP/1.0"}, + {Method: "HEAD", Status: 200, Count: 15, IP: "200.1.1.1", Protocol: "HTTP/2.0"}, + {Method: "HEAD", Status: 200, Count: 4, IP: "200.1.1.1", Protocol: "HTTP/1.0"}, + {Method: "HEAD", Status: 200, Count: 1, IP: "236.7.233.166", Protocol: "HTTP/2.0"}, + {Method: "HEAD", Status: 500, Count: 2, IP: "200.1.1.1", Protocol: "HTTP/2.0"}, + {Method: "PUT", Status: 200, Count: 20, IP: "200.1.1.1", Protocol: "HTTP/2.0"}, + } { + for i := 0; i < b.Count; i++ { + now = now.Add(time.Millisecond * 120) + lines = append(lines, httpLog{ + Time: now, + Method: b.Method, + Status: b.Status, + Bytes: 250, + Protocol: b.Protocol, + IP: b.IP, + URL: "/api/v1/series", + Ref: "https://api.go-faster.org", + }) + } + } + + for _, l := range lines { + if err := l.Append(&s); err != nil { + return s, err + } + } + return s, nil +} + +func TestGenerateLogs(t *testing.T) { + now := time.Date(2021, 1, 1, 0, 0, 0, 0, time.UTC) + set, err := generateLogs(now) + require.NoError(t, err) + logEncoder := plog.JSONMarshaler{} + var out bytes.Buffer + for _, b := range set.Batches { + data, err := logEncoder.MarshalLogs(b) + require.NoError(t, err) + outData, err := yaml.JSONToYAML(data) + require.NoError(t, err) + out.WriteString("---\n") + out.Write(outData) + } + + gold.Str(t, out.String(), "logs.yml") +} diff --git a/integration/lokie2e/loki_e2e.go b/integration/lokie2e/loki_e2e.go index 0dda10e9..f24ea6d3 100644 --- a/integration/lokie2e/loki_e2e.go +++ b/integration/lokie2e/loki_e2e.go @@ -2,10 +2,7 @@ package lokie2e import ( - "io" - "github.com/go-faster/errors" - "github.com/go-faster/jx" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/plog" @@ -23,30 +20,7 @@ type BatchSet struct { End otelstorage.Timestamp } -// ParseBatchSet parses JSON batches from given reader. -func ParseBatchSet(r io.Reader) (s BatchSet, _ error) { - d := jx.Decode(r, 4096) - u := plog.JSONUnmarshaler{} - - for d.Next() != jx.Invalid { - data, err := d.Raw() - if err != nil { - return s, errors.Wrap(err, "read line") - } - - raw, err := u.UnmarshalLogs(data) - if err != nil { - return s, errors.Wrap(err, "parse batch") - } - - if err := s.addBatch(raw); err != nil { - return s, errors.Wrap(err, "add batch") - } - } - return s, nil -} - -func (s *BatchSet) addBatch(raw plog.Logs) error { +func (s *BatchSet) Append(raw plog.Logs) error { s.Batches = append(s.Batches, raw) resLogs := raw.ResourceLogs() diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index ff412767..6d5ac749 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -262,7 +262,6 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta if err != nil { return nil, errors.Wrap(err, "get label mapping") } - fmt.Println("mapping:", mapping) out := newLogColumns() var query strings.Builder @@ -277,10 +276,7 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta for _, m := range params.Labels { labelName := string(m.Label) if key, ok := mapping[labelName]; ok { - fmt.Println("mapped", labelName, "to", key) labelName = key - } else { - fmt.Println("no mapping for", labelName) } switch m.Op { case logql.OpEq, logql.OpRe: diff --git a/internal/chstorage/schema_logs.go b/internal/chstorage/schema_logs.go index 48fb2e85..d4110f34 100644 --- a/internal/chstorage/schema_logs.go +++ b/internal/chstorage/schema_logs.go @@ -31,9 +31,15 @@ CREATE TABLE IF NOT EXISTS %s scope_name LowCardinality(String), scope_version LowCardinality(String), scope_attributes String, -- json object + + -- for selects by trace_id, span_id to discover service_name, service_namespace and timestamp + -- like SELECT service_name, service_namespace, timestamp FROM logs WHERE trace_id = '...' + -- probably can be aggregated/grouped + PROJECTION tracing (SELECT service_namespace, service_name, timestamp, trace_id, span_id ORDER BY trace_id, span_id) ) - ENGINE = MergeTree() - ORDER BY (service_namespace, service_name, service_instance_id, toStartOfFiveMinutes(timestamp));` + ENGINE = MergeTree + PRIMARY KEY (service_namespace, service_name, toStartOfFiveMinutes(timestamp)) + ORDER BY (service_namespace, service_name, toStartOfFiveMinutes(timestamp), timestamp);` logAttrsSchema = ` CREATE TABLE IF NOT EXISTS %s diff --git a/internal/logql/logqlengine/eval_streams.go b/internal/logql/logqlengine/eval_streams.go index dae46b6a..38e20995 100644 --- a/internal/logql/logqlengine/eval_streams.go +++ b/internal/logql/logqlengine/eval_streams.go @@ -42,7 +42,7 @@ func (i *entryIterator) Next(e *entry) bool { ts := record.Timestamp e.set.SetFromRecord(record) - line, keep := i.prefilter.Process(ts, record.Body, e.set) + line, keep := i.prefilter.Process(ts, LineFromRecord(record), e.set) if !keep { continue } diff --git a/internal/logql/logqlengine/json.go b/internal/logql/logqlengine/json.go index d0aab75f..ff70f957 100644 --- a/internal/logql/logqlengine/json.go +++ b/internal/logql/logqlengine/json.go @@ -112,7 +112,7 @@ func extractAll(line string, set LabelSet) error { return nil } // TODO(tdakkota): try string interning - set.Set(logql.Label(key), value) + set.Set(logql.Label(otelstorage.KeyToLabel(key)), value) return nil }) } diff --git a/internal/logql/logqlengine/label_set.go b/internal/logql/logqlengine/label_set.go index 8681fb50..5170208c 100644 --- a/internal/logql/logqlengine/label_set.go +++ b/internal/logql/logqlengine/label_set.go @@ -80,16 +80,13 @@ func (l *LabelSet) SetFromRecord(record logstorage.Record) { l.reset() if traceID := record.TraceID; !traceID.IsEmpty() { - l.Set(logql.Label(`trace_id`), pcommon.NewValueStr(traceID.Hex())) + l.Set(`trace_id`, pcommon.NewValueStr(traceID.Hex())) } if spanID := record.SpanID; !spanID.IsEmpty() { - l.Set(logql.Label(`span_id`), pcommon.NewValueStr(spanID.Hex())) - } - if severity := record.SeverityText; severity != "" { - l.Set(logql.Label(`severity_text`), pcommon.NewValueStr(severity)) + l.Set(`span_id`, pcommon.NewValueStr(spanID.Hex())) } if severity := record.SeverityNumber; severity != plog.SeverityNumberUnspecified { - l.Set(logql.Label(`severity_number`), pcommon.NewValueInt(int64(severity))) + l.Set(`level`, pcommon.NewValueStr(severity.String())) } l.SetAttrs(record.Attrs, record.ScopeAttrs, record.ResourceAttrs) } @@ -102,10 +99,7 @@ func (l *LabelSet) SetAttrs(attrMaps ...otelstorage.Attrs) { continue } m.Range(func(k string, v pcommon.Value) bool { - if err := logql.IsValidLabel(k, l.allowDots()); err != nil { - l.SetError("record extraction", err) - return false - } + k = otelstorage.KeyToLabel(k) l.Set(logql.Label(k), v) return true }) diff --git a/internal/logql/logqlengine/otel_adapter.go b/internal/logql/logqlengine/otel_adapter.go new file mode 100644 index 00000000..6c06669b --- /dev/null +++ b/internal/logql/logqlengine/otel_adapter.go @@ -0,0 +1,38 @@ +package logqlengine + +import ( + "github.com/go-faster/jx" + "go.opentelemetry.io/collector/pdata/pcommon" + + "github.com/go-faster/oteldb/internal/logstorage" +) + +// LineFromRecord returns a JSON line from a log record. +func LineFromRecord(record logstorage.Record) string { + // Create JSON object from record. + e := &jx.Encoder{} + e.Obj(func(e *jx.Encoder) { + e.Field("msg", func(e *jx.Encoder) { + e.Str(record.Body) + }) + record.Attrs.AsMap().Range(func(k string, v pcommon.Value) bool { + e.Field(k, func(e *jx.Encoder) { + switch v.Type() { + case pcommon.ValueTypeStr: + e.Str(v.Str()) + case pcommon.ValueTypeBool: + e.Bool(v.Bool()) + case pcommon.ValueTypeInt: + e.Int64(v.Int()) + case pcommon.ValueTypeDouble: + e.Float64(v.Double()) + default: + // Fallback. + e.Str(v.AsString()) + } + }) + return true + }) + }) + return e.String() +} From c2118ec02a7a22d68cb7e36b8f93e3f8f7be8dd3 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 15:15:24 +0300 Subject: [PATCH 093/112] chore: rm --- integration/lokie2e/common_test.go | 7 ------- 1 file changed, 7 deletions(-) diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index 871e8129..b0a9526e 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -130,13 +130,6 @@ func runTest( } }) t.Run("LogQueries", func(t *testing.T) { - // Example JQ expression to make testdata queries: - // - // .resourceLogs[].scopeLogs[].logRecords[] - // | .body.stringValue - // | fromjson - // | select(.method=="GET") - // tests := []struct { query string entries int From 73b1e67a795a8f58c6f2061f7b12181e7de080a5 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 15:15:48 +0300 Subject: [PATCH 094/112] chore(git): upd submodule --- internal/otelschema/_testdata/semantic-conventions | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/otelschema/_testdata/semantic-conventions b/internal/otelschema/_testdata/semantic-conventions index fe200c01..efd93451 160000 --- a/internal/otelschema/_testdata/semantic-conventions +++ b/internal/otelschema/_testdata/semantic-conventions @@ -1 +1 @@ -Subproject commit fe200c01bcf589b30f2ae9b81152d69252d8ea21 +Subproject commit efd9345199e986e61488def054a33f193acf7298 From c8f58df554479a34303063d788bfa83e7231aaad Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 15:19:29 +0300 Subject: [PATCH 095/112] test(lokie2e): also test with dots --- integration/lokie2e/common_test.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index b0a9526e..eb618fe9 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -140,6 +140,11 @@ func runTest( {`{http_method=~".*GET.*"}`, 21}, {`{http_method=~"^GET$"}`, 21}, {`{http_method!~"(HEAD|POST|DELETE|PUT|PATCH|TRACE|OPTIONS)"}`, 21}, + // Also with dots. + {`{http.method="GET"}`, 21}, + {`{http.method=~".*GET.*"}`, 21}, + {`{http.method=~"^GET$"}`, 21}, + {`{http.method!~"(HEAD|POST|DELETE|PUT|PATCH|TRACE|OPTIONS)"}`, 21}, // Try other methods. {`{http_method="DELETE"}`, 20}, {`{http_method="GET"}`, 21}, @@ -155,6 +160,10 @@ func runTest( {`{http_method="HEAD",http_status_code="500"}`, 2}, {`{http_method="HEAD",http_status_code=~"^500$"}`, 2}, {`{http_method=~".*HEAD.*",http_status_code=~"^500$"}`, 2}, + // Also with dots. + {`{http.method="HEAD",http.status_code="500"}`, 2}, + {`{http.method="HEAD",http.status_code=~"^500$"}`, 2}, + {`{http.method=~".*HEAD.*",http.status_code=~"^500$"}`, 2}, // Line filter. {`{http_method=~".+"} |= "GET"`, 21}, From e584ca53448fc3af0bcf0bc3e2ffa3f38d5a096b Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 16:35:46 +0300 Subject: [PATCH 096/112] fix(chstorage.querier.logs): correctly return label values --- integration/lokie2e/common_test.go | 37 +++++++++++++++++++----------- internal/chstorage/querier_logs.go | 22 ++++++++++-------- 2 files changed, 36 insertions(+), 23 deletions(-) diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index eb618fe9..365f3b77 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -6,6 +6,7 @@ import ( "fmt" "net/http/httptest" "os" + "slices" "testing" "time" @@ -108,25 +109,33 @@ func runTest( } }) t.Run("LabelValues", func(t *testing.T) { - a := require.New(t) - for labelName, labels := range set.Labels { - labelValue := map[string]struct{}{} + unique := map[string]struct{}{} for _, t := range labels { - labelValue[t.Value] = struct{}{} + unique[t.Value] = struct{}{} + } + values := make([]string, 0, len(unique)) + for v := range unique { + values = append(values, v) } + slices.Sort(values) - r, err := c.LabelValues(ctx, lokiapi.LabelValuesParams{ - Name: labelName, - // Always sending time range because default is current time. - Start: lokiapi.NewOptLokiTime(asLokiTime(set.Start)), - End: lokiapi.NewOptLokiTime(asLokiTime(set.End)), + t.Run(labelName, func(t *testing.T) { + a := require.New(t) + t.Logf("%q: %v", labelName, values) + r, err := c.LabelValues(ctx, lokiapi.LabelValuesParams{ + Name: labelName, + // Always sending time range because default is current time. + Start: lokiapi.NewOptLokiTime(asLokiTime(set.Start)), + End: lokiapi.NewOptLokiTime(asLokiTime(set.End)), + }) + a.NoError(err) + a.Len(r.Data, len(values)) + t.Logf("got values %v", r.Data) + for _, val := range r.Data { + a.Containsf(values, val, "check label %q", labelName) + } }) - a.NoError(err) - a.Len(r.Data, len(labelValue)) - for _, val := range r.Data { - a.Containsf(labelValue, val, "check label %q", labelName) - } } }) t.Run("LogQueries", func(t *testing.T) { diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index 6d5ac749..5ea38d33 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -177,27 +177,31 @@ func (q *Querier) LabelValues(ctx context.Context, labelName string, opts logsto labelName = key } } - var ( - names proto.ColStr - out []jx.Raw - ) + var out []jx.Raw + values := new(proto.ColStr).Array() if err := q.ch.Do(ctx, ch.Query{ Logger: zctx.From(ctx).Named("ch"), Result: proto.Results{ - {Name: "value", Data: &names}, + {Name: "values", Data: values}, }, OnResult: func(ctx context.Context, block proto.Block) error { - for i := 0; i < names.Rows(); i++ { - out = append(out, jx.Raw(names.Row(i))) + for i := 0; i < values.Rows(); i++ { + for _, v := range values.Row(i) { + if len(v) == 0 { + // HACK: JSONExtractRaw returns empty string if key is not found. + continue + } + out = append(out, jx.Raw(v)) + } } return nil }, Body: fmt.Sprintf(`SELECT DISTINCT -COALESCE( +array( JSONExtractRaw(attributes, %[1]s), JSONExtractRaw(scope_attributes, %[1]s), JSONExtractRaw(resource, %[1]s) -) as value +) as values FROM %s WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d) LIMIT 1000`, singleQuoted(labelName), table, opts.Start, opts.End, From 8cb79fe28288c14ecc4e2af051cebcf178799f1a Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 16:48:35 +0300 Subject: [PATCH 097/112] feat(logqlengine): add msg to labels This will allow to "show this field" toggle in Grafana UI. --- internal/logql/logqlengine/label_set.go | 1 + 1 file changed, 1 insertion(+) diff --git a/internal/logql/logqlengine/label_set.go b/internal/logql/logqlengine/label_set.go index 5170208c..609948c3 100644 --- a/internal/logql/logqlengine/label_set.go +++ b/internal/logql/logqlengine/label_set.go @@ -88,6 +88,7 @@ func (l *LabelSet) SetFromRecord(record logstorage.Record) { if severity := record.SeverityNumber; severity != plog.SeverityNumberUnspecified { l.Set(`level`, pcommon.NewValueStr(severity.String())) } + l.Set(`msg`, pcommon.NewValueStr(record.Body)) l.SetAttrs(record.Attrs, record.ScopeAttrs, record.ResourceAttrs) } From a6d31e4958e189ffe1ff42f056036ca1e5f03f14 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 17:41:30 +0300 Subject: [PATCH 098/112] docs(readme): upd --- README.md | 21 ++++++++------------- 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/README.md b/README.md index 60e7f81e..10675b04 100644 --- a/README.md +++ b/README.md @@ -8,30 +8,25 @@ The OpenTelemetry-compatible telemetry aggregation, storage and processing. Work in progress. +> [!WARNING] +> Work in progress. + ## Storage The oteldb is stateless and uses external storage systems for data persistence, processing and aggregation. -### YTsaurus - -The [YTsaurus](https://ytsaurus.tech/) is a primary storage for telemetry data. -An open source big data platform for distributed storage and processing. - -- Hierarchical multi-tenancy with secure resource isolation -- OLAP and OLTP -- MapReduce, ACID -- ClickHouse protocol compatible -- Exabyte scale, up to 1M CPU, 10K+ nodes - -### ClickHouse +We focus on the following storage systems: +- [ClickHouse](https://clickhouse.com/) +- [YTsaurus](https://ytsaurus.tech/) -The oteldb also supports [ClickHouse](https://clickhouse.com/) storage. +Currently, ClickHouse looks more promising. ## Query Supported query languages: - LogQL (loki) for logs - TraceQL (Tempo) for traces +- PromQL (Prometheus) for metrics ## Local development From 0ade99e2f131bf194d0c6e560009ef76ae0e51d9 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 19:17:21 +0300 Subject: [PATCH 099/112] feat(logql): improve loki schema mapping Update logqlengine, chstorage, lokie2e. This improves materialized fields handling. --- integration/lokie2e/_golden/logs.yml | 615 ++++++++++++--------- integration/lokie2e/common_test.go | 21 +- integration/lokie2e/generate_test.go | 44 +- integration/lokie2e/loki_e2e.go | 34 ++ internal/chstorage/querier_logs.go | 140 ++++- internal/logql/logqlengine/label_set.go | 8 +- internal/logql/logqlengine/otel_adapter.go | 16 +- internal/logstorage/schema.go | 12 + internal/otelschema/_golden/ddl.sql | 2 + internal/otelschema/_golden/registry.yaml | 22 +- 10 files changed, 642 insertions(+), 272 deletions(-) diff --git a/integration/lokie2e/_golden/logs.yml b/integration/lokie2e/_golden/logs.yml index 9a624a70..70bb53fc 100644 --- a/integration/lokie2e/_golden/logs.yml +++ b/integration/lokie2e/_golden/logs.yml @@ -31,9 +31,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459200120000000" - spanId: "" + severityNumber: 9 + spanId: b7d8ccf703000000 timeUnixNano: "1609459200120000000" - traceId: "" + traceId: f71e0000000000000d00000000000000 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -76,9 +77,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459200240000000" - spanId: "" + severityNumber: 9 + spanId: b8d8ccf703000100 timeUnixNano: "1609459200240000000" - traceId: "" + traceId: f81e0000000000000e00000000000100 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -121,9 +123,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459200360000000" - spanId: "" + severityNumber: 9 + spanId: b9d8ccf703000200 timeUnixNano: "1609459200360000000" - traceId: "" + traceId: f91e0000000000000f00000000000200 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -166,9 +169,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459200480000000" - spanId: "" + severityNumber: 9 + spanId: bad8ccf703000300 timeUnixNano: "1609459200480000000" - traceId: "" + traceId: fa1e0000000000001000000000000300 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -211,9 +215,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459200600000000" - spanId: "" + severityNumber: 9 + spanId: bbd8ccf703000400 timeUnixNano: "1609459200600000000" - traceId: "" + traceId: fb1e0000000000001100000000000400 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -256,9 +261,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459200720000000" - spanId: "" + severityNumber: 9 + spanId: bcd8ccf703000500 timeUnixNano: "1609459200720000000" - traceId: "" + traceId: fc1e0000000000001200000000000500 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -301,9 +307,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459200840000000" - spanId: "" + severityNumber: 9 + spanId: bdd8ccf703000600 timeUnixNano: "1609459200840000000" - traceId: "" + traceId: fd1e0000000000001300000000000600 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -346,9 +353,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459200960000000" - spanId: "" + severityNumber: 9 + spanId: bed8ccf703000700 timeUnixNano: "1609459200960000000" - traceId: "" + traceId: fe1e0000000000001400000000000700 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -391,9 +399,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459201080000000" - spanId: "" + severityNumber: 9 + spanId: bfd8ccf703000800 timeUnixNano: "1609459201080000000" - traceId: "" + traceId: ff1e0000000000001500000000000800 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -436,9 +445,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459201200000000" - spanId: "" + severityNumber: 9 + spanId: c0d8ccf703000900 timeUnixNano: "1609459201200000000" - traceId: "" + traceId: 801f0000000000001600000000000900 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -481,9 +491,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459201320000000" - spanId: "" + severityNumber: 9 + spanId: c1d8ccf703000a00 timeUnixNano: "1609459201320000000" - traceId: "" + traceId: 811f0000000000001700000000000a00 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -526,9 +537,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459201440000000" - spanId: "" + severityNumber: 9 + spanId: 9bd9ccf703000001 timeUnixNano: "1609459201440000000" - traceId: "" + traceId: df26000000000000f507000000000001 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -571,9 +583,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459201560000000" - spanId: "" + severityNumber: 9 + spanId: 9cd9ccf703000101 timeUnixNano: "1609459201560000000" - traceId: "" + traceId: e026000000000000f607000000000101 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -616,9 +629,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459201680000000" - spanId: "" + severityNumber: 9 + spanId: 9dd9ccf703000201 timeUnixNano: "1609459201680000000" - traceId: "" + traceId: e126000000000000f707000000000201 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -661,9 +675,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459201800000000" - spanId: "" + severityNumber: 9 + spanId: 9ed9ccf703000301 timeUnixNano: "1609459201800000000" - traceId: "" + traceId: e226000000000000f807000000000301 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -706,9 +721,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459201920000000" - spanId: "" + severityNumber: 9 + spanId: 9fd9ccf703000401 timeUnixNano: "1609459201920000000" - traceId: "" + traceId: e326000000000000f907000000000401 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -751,9 +767,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459202040000000" - spanId: "" + severityNumber: 9 + spanId: a0d9ccf703000501 timeUnixNano: "1609459202040000000" - traceId: "" + traceId: e426000000000000fa07000000000501 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -796,9 +813,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459202160000000" - spanId: "" + severityNumber: 9 + spanId: a1d9ccf703000601 timeUnixNano: "1609459202160000000" - traceId: "" + traceId: e526000000000000fb07000000000601 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -841,9 +859,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459202280000000" - spanId: "" + severityNumber: 9 + spanId: a2d9ccf703000701 timeUnixNano: "1609459202280000000" - traceId: "" + traceId: e626000000000000fc07000000000701 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -886,9 +905,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459202400000000" - spanId: "" + severityNumber: 9 + spanId: a3d9ccf703000801 timeUnixNano: "1609459202400000000" - traceId: "" + traceId: e726000000000000fd07000000000801 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -931,9 +951,10 @@ resourceLogs: stringValue: GET /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459202520000000" - spanId: "" + severityNumber: 9 + spanId: a4d9ccf703000901 timeUnixNano: "1609459202520000000" - traceId: "" + traceId: e826000000000000fe07000000000901 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -976,9 +997,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459202640000000" - spanId: "" + severityNumber: 9 + spanId: ffd9ccf703000002 timeUnixNano: "1609459202640000000" - traceId: "" + traceId: c72e000000000000dd0f000000000002 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1021,9 +1043,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459202760000000" - spanId: "" + severityNumber: 9 + spanId: 80daccf703000102 timeUnixNano: "1609459202760000000" - traceId: "" + traceId: c82e000000000000de0f000000000102 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1066,9 +1089,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459202880000000" - spanId: "" + severityNumber: 9 + spanId: 81daccf703000202 timeUnixNano: "1609459202880000000" - traceId: "" + traceId: c92e000000000000df0f000000000202 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1111,9 +1135,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203000000000" - spanId: "" + severityNumber: 9 + spanId: 82daccf703000302 timeUnixNano: "1609459203000000000" - traceId: "" + traceId: ca2e000000000000e00f000000000302 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1156,9 +1181,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203120000000" - spanId: "" + severityNumber: 9 + spanId: 83daccf703000402 timeUnixNano: "1609459203120000000" - traceId: "" + traceId: cb2e000000000000e10f000000000402 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1201,9 +1227,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203240000000" - spanId: "" + severityNumber: 9 + spanId: 84daccf703000502 timeUnixNano: "1609459203240000000" - traceId: "" + traceId: cc2e000000000000e20f000000000502 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1246,9 +1273,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203360000000" - spanId: "" + severityNumber: 9 + spanId: 85daccf703000602 timeUnixNano: "1609459203360000000" - traceId: "" + traceId: cd2e000000000000e30f000000000602 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1291,9 +1319,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203480000000" - spanId: "" + severityNumber: 9 + spanId: 86daccf703000702 timeUnixNano: "1609459203480000000" - traceId: "" + traceId: ce2e000000000000e40f000000000702 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1336,9 +1365,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203600000000" - spanId: "" + severityNumber: 9 + spanId: 87daccf703000802 timeUnixNano: "1609459203600000000" - traceId: "" + traceId: cf2e000000000000e50f000000000802 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1381,9 +1411,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203720000000" - spanId: "" + severityNumber: 9 + spanId: 88daccf703000902 timeUnixNano: "1609459203720000000" - traceId: "" + traceId: d02e000000000000e60f000000000902 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1426,9 +1457,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203840000000" - spanId: "" + severityNumber: 9 + spanId: 89daccf703000a02 timeUnixNano: "1609459203840000000" - traceId: "" + traceId: d12e000000000000e70f000000000a02 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1471,9 +1503,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459203960000000" - spanId: "" + severityNumber: 9 + spanId: 8adaccf703000b02 timeUnixNano: "1609459203960000000" - traceId: "" + traceId: d22e000000000000e80f000000000b02 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1516,9 +1549,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459204080000000" - spanId: "" + severityNumber: 9 + spanId: 8bdaccf703000c02 timeUnixNano: "1609459204080000000" - traceId: "" + traceId: d32e000000000000e90f000000000c02 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1561,9 +1595,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459204200000000" - spanId: "" + severityNumber: 9 + spanId: 8cdaccf703000d02 timeUnixNano: "1609459204200000000" - traceId: "" + traceId: d42e000000000000ea0f000000000d02 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1606,9 +1641,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459204320000000" - spanId: "" + severityNumber: 9 + spanId: 8ddaccf703000e02 timeUnixNano: "1609459204320000000" - traceId: "" + traceId: d52e000000000000eb0f000000000e02 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1651,9 +1687,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459204440000000" - spanId: "" + severityNumber: 9 + spanId: 8edaccf703000f02 timeUnixNano: "1609459204440000000" - traceId: "" + traceId: d62e000000000000ec0f000000000f02 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1696,9 +1733,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459204560000000" - spanId: "" + severityNumber: 9 + spanId: 8fdaccf703001002 timeUnixNano: "1609459204560000000" - traceId: "" + traceId: d72e000000000000ed0f000000001002 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1741,9 +1779,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459204680000000" - spanId: "" + severityNumber: 9 + spanId: 90daccf703001102 timeUnixNano: "1609459204680000000" - traceId: "" + traceId: d82e000000000000ee0f000000001102 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1786,9 +1825,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459204800000000" - spanId: "" + severityNumber: 9 + spanId: 91daccf703001202 timeUnixNano: "1609459204800000000" - traceId: "" + traceId: d92e000000000000ef0f000000001202 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1831,9 +1871,10 @@ resourceLogs: stringValue: DELETE /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459204920000000" - spanId: "" + severityNumber: 9 + spanId: 92daccf703001302 timeUnixNano: "1609459204920000000" - traceId: "" + traceId: da2e000000000000f00f000000001302 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1876,9 +1917,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459205040000000" - spanId: "" + severityNumber: 9 + spanId: e3daccf703000003 timeUnixNano: "1609459205040000000" - traceId: "" + traceId: af36000000000000c517000000000003 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1921,9 +1963,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459205160000000" - spanId: "" + severityNumber: 9 + spanId: e4daccf703000103 timeUnixNano: "1609459205160000000" - traceId: "" + traceId: b036000000000000c617000000000103 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -1966,9 +2009,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459205280000000" - spanId: "" + severityNumber: 9 + spanId: e5daccf703000203 timeUnixNano: "1609459205280000000" - traceId: "" + traceId: b136000000000000c717000000000203 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2011,9 +2055,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459205400000000" - spanId: "" + severityNumber: 9 + spanId: e6daccf703000303 timeUnixNano: "1609459205400000000" - traceId: "" + traceId: b236000000000000c817000000000303 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2056,9 +2101,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459205520000000" - spanId: "" + severityNumber: 9 + spanId: e7daccf703000403 timeUnixNano: "1609459205520000000" - traceId: "" + traceId: b336000000000000c917000000000403 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2101,9 +2147,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459205640000000" - spanId: "" + severityNumber: 9 + spanId: e8daccf703000503 timeUnixNano: "1609459205640000000" - traceId: "" + traceId: b436000000000000ca17000000000503 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2146,9 +2193,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459205760000000" - spanId: "" + severityNumber: 9 + spanId: e9daccf703000603 timeUnixNano: "1609459205760000000" - traceId: "" + traceId: b536000000000000cb17000000000603 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2191,9 +2239,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459205880000000" - spanId: "" + severityNumber: 9 + spanId: eadaccf703000703 timeUnixNano: "1609459205880000000" - traceId: "" + traceId: b636000000000000cc17000000000703 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2236,9 +2285,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206000000000" - spanId: "" + severityNumber: 9 + spanId: ebdaccf703000803 timeUnixNano: "1609459206000000000" - traceId: "" + traceId: b736000000000000cd17000000000803 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2281,9 +2331,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206120000000" - spanId: "" + severityNumber: 9 + spanId: ecdaccf703000903 timeUnixNano: "1609459206120000000" - traceId: "" + traceId: b836000000000000ce17000000000903 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2326,9 +2377,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206240000000" - spanId: "" + severityNumber: 9 + spanId: eddaccf703000a03 timeUnixNano: "1609459206240000000" - traceId: "" + traceId: b936000000000000cf17000000000a03 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2371,9 +2423,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206360000000" - spanId: "" + severityNumber: 9 + spanId: eedaccf703000b03 timeUnixNano: "1609459206360000000" - traceId: "" + traceId: ba36000000000000d017000000000b03 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2416,9 +2469,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206480000000" - spanId: "" + severityNumber: 9 + spanId: efdaccf703000c03 timeUnixNano: "1609459206480000000" - traceId: "" + traceId: bb36000000000000d117000000000c03 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2461,9 +2515,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206600000000" - spanId: "" + severityNumber: 9 + spanId: f0daccf703000d03 timeUnixNano: "1609459206600000000" - traceId: "" + traceId: bc36000000000000d217000000000d03 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2506,9 +2561,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206720000000" - spanId: "" + severityNumber: 9 + spanId: f1daccf703000e03 timeUnixNano: "1609459206720000000" - traceId: "" + traceId: bd36000000000000d317000000000e03 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2551,9 +2607,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206840000000" - spanId: "" + severityNumber: 9 + spanId: f2daccf703000f03 timeUnixNano: "1609459206840000000" - traceId: "" + traceId: be36000000000000d417000000000f03 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2596,9 +2653,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459206960000000" - spanId: "" + severityNumber: 9 + spanId: f3daccf703001003 timeUnixNano: "1609459206960000000" - traceId: "" + traceId: bf36000000000000d517000000001003 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2641,9 +2699,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459207080000000" - spanId: "" + severityNumber: 9 + spanId: f4daccf703001103 timeUnixNano: "1609459207080000000" - traceId: "" + traceId: c036000000000000d617000000001103 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2686,9 +2745,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459207200000000" - spanId: "" + severityNumber: 9 + spanId: f5daccf703001203 timeUnixNano: "1609459207200000000" - traceId: "" + traceId: c136000000000000d717000000001203 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2731,9 +2791,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459207320000000" - spanId: "" + severityNumber: 9 + spanId: f6daccf703001303 timeUnixNano: "1609459207320000000" - traceId: "" + traceId: c236000000000000d817000000001303 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2776,9 +2837,10 @@ resourceLogs: stringValue: POST /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459207440000000" - spanId: "" + severityNumber: 9 + spanId: f7daccf703001403 timeUnixNano: "1609459207440000000" - traceId: "" + traceId: c336000000000000d917000000001403 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2821,9 +2883,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459207560000000" - spanId: "" + severityNumber: 9 + spanId: c7dbccf703000004 timeUnixNano: "1609459207560000000" - traceId: "" + traceId: 973e000000000000ad1f000000000004 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2866,9 +2929,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459207680000000" - spanId: "" + severityNumber: 9 + spanId: c8dbccf703000104 timeUnixNano: "1609459207680000000" - traceId: "" + traceId: 983e000000000000ae1f000000000104 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2911,9 +2975,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459207800000000" - spanId: "" + severityNumber: 9 + spanId: c9dbccf703000204 timeUnixNano: "1609459207800000000" - traceId: "" + traceId: 993e000000000000af1f000000000204 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -2956,9 +3021,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459207920000000" - spanId: "" + severityNumber: 9 + spanId: cadbccf703000304 timeUnixNano: "1609459207920000000" - traceId: "" + traceId: 9a3e000000000000b01f000000000304 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3001,9 +3067,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459208040000000" - spanId: "" + severityNumber: 9 + spanId: cbdbccf703000404 timeUnixNano: "1609459208040000000" - traceId: "" + traceId: 9b3e000000000000b11f000000000404 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3046,9 +3113,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459208160000000" - spanId: "" + severityNumber: 9 + spanId: ccdbccf703000504 timeUnixNano: "1609459208160000000" - traceId: "" + traceId: 9c3e000000000000b21f000000000504 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3091,9 +3159,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459208280000000" - spanId: "" + severityNumber: 9 + spanId: cddbccf703000604 timeUnixNano: "1609459208280000000" - traceId: "" + traceId: 9d3e000000000000b31f000000000604 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3136,9 +3205,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459208400000000" - spanId: "" + severityNumber: 9 + spanId: cedbccf703000704 timeUnixNano: "1609459208400000000" - traceId: "" + traceId: 9e3e000000000000b41f000000000704 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3181,9 +3251,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459208520000000" - spanId: "" + severityNumber: 9 + spanId: cfdbccf703000804 timeUnixNano: "1609459208520000000" - traceId: "" + traceId: 9f3e000000000000b51f000000000804 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3226,9 +3297,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459208640000000" - spanId: "" + severityNumber: 9 + spanId: d0dbccf703000904 timeUnixNano: "1609459208640000000" - traceId: "" + traceId: a03e000000000000b61f000000000904 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3271,9 +3343,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459208760000000" - spanId: "" + severityNumber: 9 + spanId: d1dbccf703000a04 timeUnixNano: "1609459208760000000" - traceId: "" + traceId: a13e000000000000b71f000000000a04 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3316,9 +3389,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459208880000000" - spanId: "" + severityNumber: 9 + spanId: d2dbccf703000b04 timeUnixNano: "1609459208880000000" - traceId: "" + traceId: a23e000000000000b81f000000000b04 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3361,9 +3435,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209000000000" - spanId: "" + severityNumber: 9 + spanId: d3dbccf703000c04 timeUnixNano: "1609459209000000000" - traceId: "" + traceId: a33e000000000000b91f000000000c04 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3406,9 +3481,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209120000000" - spanId: "" + severityNumber: 9 + spanId: d4dbccf703000d04 timeUnixNano: "1609459209120000000" - traceId: "" + traceId: a43e000000000000ba1f000000000d04 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3451,9 +3527,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209240000000" - spanId: "" + severityNumber: 9 + spanId: d5dbccf703000e04 timeUnixNano: "1609459209240000000" - traceId: "" + traceId: a53e000000000000bb1f000000000e04 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3496,9 +3573,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209360000000" - spanId: "" + severityNumber: 9 + spanId: d6dbccf703000f04 timeUnixNano: "1609459209360000000" - traceId: "" + traceId: a63e000000000000bc1f000000000f04 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3541,9 +3619,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209480000000" - spanId: "" + severityNumber: 9 + spanId: d7dbccf703001004 timeUnixNano: "1609459209480000000" - traceId: "" + traceId: a73e000000000000bd1f000000001004 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3586,9 +3665,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209600000000" - spanId: "" + severityNumber: 9 + spanId: d8dbccf703001104 timeUnixNano: "1609459209600000000" - traceId: "" + traceId: a83e000000000000be1f000000001104 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3631,9 +3711,10 @@ resourceLogs: stringValue: PATCH /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209720000000" - spanId: "" + severityNumber: 9 + spanId: d9dbccf703001204 timeUnixNano: "1609459209720000000" - traceId: "" + traceId: a93e000000000000bf1f000000001204 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3676,9 +3757,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209840000000" - spanId: "" + severityNumber: 9 + spanId: abdcccf703000005 timeUnixNano: "1609459209840000000" - traceId: "" + traceId: ff450000000000009527000000000005 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3721,9 +3803,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459209960000000" - spanId: "" + severityNumber: 9 + spanId: acdcccf703000105 timeUnixNano: "1609459209960000000" - traceId: "" + traceId: "80460000000000009627000000000105" schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3766,9 +3849,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459210080000000" - spanId: "" + severityNumber: 9 + spanId: addcccf703000205 timeUnixNano: "1609459210080000000" - traceId: "" + traceId: "81460000000000009727000000000205" schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3811,9 +3895,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459210200000000" - spanId: "" + severityNumber: 9 + spanId: aedcccf703000305 timeUnixNano: "1609459210200000000" - traceId: "" + traceId: "82460000000000009827000000000305" schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3856,9 +3941,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459210320000000" - spanId: "" + severityNumber: 9 + spanId: afdcccf703000405 timeUnixNano: "1609459210320000000" - traceId: "" + traceId: "83460000000000009927000000000405" schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3901,9 +3987,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459210440000000" - spanId: "" + severityNumber: 9 + spanId: b0dcccf703000505 timeUnixNano: "1609459210440000000" - traceId: "" + traceId: 84460000000000009a27000000000505 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3946,9 +4033,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459210560000000" - spanId: "" + severityNumber: 9 + spanId: b1dcccf703000605 timeUnixNano: "1609459210560000000" - traceId: "" + traceId: 85460000000000009b27000000000605 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -3991,9 +4079,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459210680000000" - spanId: "" + severityNumber: 9 + spanId: b2dcccf703000705 timeUnixNano: "1609459210680000000" - traceId: "" + traceId: 86460000000000009c27000000000705 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4036,9 +4125,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459210800000000" - spanId: "" + severityNumber: 9 + spanId: b3dcccf703000805 timeUnixNano: "1609459210800000000" - traceId: "" + traceId: 87460000000000009d27000000000805 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4081,9 +4171,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459210920000000" - spanId: "" + severityNumber: 9 + spanId: b4dcccf703000905 timeUnixNano: "1609459210920000000" - traceId: "" + traceId: 88460000000000009e27000000000905 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4126,9 +4217,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459211040000000" - spanId: "" + severityNumber: 9 + spanId: b5dcccf703000a05 timeUnixNano: "1609459211040000000" - traceId: "" + traceId: 89460000000000009f27000000000a05 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4171,9 +4263,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459211160000000" - spanId: "" + severityNumber: 9 + spanId: b6dcccf703000b05 timeUnixNano: "1609459211160000000" - traceId: "" + traceId: 8a46000000000000a027000000000b05 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4216,9 +4309,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459211280000000" - spanId: "" + severityNumber: 9 + spanId: b7dcccf703000c05 timeUnixNano: "1609459211280000000" - traceId: "" + traceId: 8b46000000000000a127000000000c05 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4261,9 +4355,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459211400000000" - spanId: "" + severityNumber: 9 + spanId: b8dcccf703000d05 timeUnixNano: "1609459211400000000" - traceId: "" + traceId: 8c46000000000000a227000000000d05 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4306,9 +4401,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459211520000000" - spanId: "" + severityNumber: 9 + spanId: b9dcccf703000e05 timeUnixNano: "1609459211520000000" - traceId: "" + traceId: 8d46000000000000a327000000000e05 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4351,9 +4447,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459211640000000" - spanId: "" + severityNumber: 9 + spanId: 8fddccf703000006 timeUnixNano: "1609459211640000000" - traceId: "" + traceId: e74d000000000000fd2e000000000006 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4396,9 +4493,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459211760000000" - spanId: "" + severityNumber: 9 + spanId: 90ddccf703000106 timeUnixNano: "1609459211760000000" - traceId: "" + traceId: e84d000000000000fe2e000000000106 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4441,9 +4539,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459211880000000" - spanId: "" + severityNumber: 9 + spanId: 91ddccf703000206 timeUnixNano: "1609459211880000000" - traceId: "" + traceId: e94d000000000000ff2e000000000206 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4486,9 +4585,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212000000000" - spanId: "" + severityNumber: 9 + spanId: 92ddccf703000306 timeUnixNano: "1609459212000000000" - traceId: "" + traceId: ea4d000000000000802f000000000306 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4531,9 +4631,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212120000000" - spanId: "" + severityNumber: 9 + spanId: f3ddccf703000007 timeUnixNano: "1609459212120000000" - traceId: "" + traceId: cf55000000000000e536000000000007 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4576,9 +4677,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 500 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212240000000" - spanId: "" + severityNumber: 21 + spanId: d7deccf703000008 timeUnixNano: "1609459212240000000" - traceId: "" + traceId: b75d000000000000cd3e000000000008 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4621,9 +4723,10 @@ resourceLogs: stringValue: HEAD /api/v1/series 500 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212360000000" - spanId: "" + severityNumber: 21 + spanId: d8deccf703000108 timeUnixNano: "1609459212360000000" - traceId: "" + traceId: b85d000000000000ce3e000000000108 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4666,9 +4769,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212480000000" - spanId: "" + severityNumber: 9 + spanId: bbdfccf703000009 timeUnixNano: "1609459212480000000" - traceId: "" + traceId: 9f65000000000000b546000000000009 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4711,9 +4815,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212600000000" - spanId: "" + severityNumber: 9 + spanId: bcdfccf703000109 timeUnixNano: "1609459212600000000" - traceId: "" + traceId: a065000000000000b646000000000109 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4756,9 +4861,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212720000000" - spanId: "" + severityNumber: 9 + spanId: bddfccf703000209 timeUnixNano: "1609459212720000000" - traceId: "" + traceId: a165000000000000b746000000000209 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4801,9 +4907,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212840000000" - spanId: "" + severityNumber: 9 + spanId: bedfccf703000309 timeUnixNano: "1609459212840000000" - traceId: "" + traceId: a265000000000000b846000000000309 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4846,9 +4953,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459212960000000" - spanId: "" + severityNumber: 9 + spanId: bfdfccf703000409 timeUnixNano: "1609459212960000000" - traceId: "" + traceId: a365000000000000b946000000000409 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4891,9 +4999,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459213080000000" - spanId: "" + severityNumber: 9 + spanId: c0dfccf703000509 timeUnixNano: "1609459213080000000" - traceId: "" + traceId: a465000000000000ba46000000000509 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4936,9 +5045,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459213200000000" - spanId: "" + severityNumber: 9 + spanId: c1dfccf703000609 timeUnixNano: "1609459213200000000" - traceId: "" + traceId: a565000000000000bb46000000000609 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -4981,9 +5091,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459213320000000" - spanId: "" + severityNumber: 9 + spanId: c2dfccf703000709 timeUnixNano: "1609459213320000000" - traceId: "" + traceId: a665000000000000bc46000000000709 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5026,9 +5137,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459213440000000" - spanId: "" + severityNumber: 9 + spanId: c3dfccf703000809 timeUnixNano: "1609459213440000000" - traceId: "" + traceId: a765000000000000bd46000000000809 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5071,9 +5183,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459213560000000" - spanId: "" + severityNumber: 9 + spanId: c4dfccf703000909 timeUnixNano: "1609459213560000000" - traceId: "" + traceId: a865000000000000be46000000000909 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5116,9 +5229,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459213680000000" - spanId: "" + severityNumber: 9 + spanId: c5dfccf703000a09 timeUnixNano: "1609459213680000000" - traceId: "" + traceId: a965000000000000bf46000000000a09 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5161,9 +5275,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459213800000000" - spanId: "" + severityNumber: 9 + spanId: c6dfccf703000b09 timeUnixNano: "1609459213800000000" - traceId: "" + traceId: aa65000000000000c046000000000b09 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5206,9 +5321,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459213920000000" - spanId: "" + severityNumber: 9 + spanId: c7dfccf703000c09 timeUnixNano: "1609459213920000000" - traceId: "" + traceId: ab65000000000000c146000000000c09 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5251,9 +5367,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459214040000000" - spanId: "" + severityNumber: 9 + spanId: c8dfccf703000d09 timeUnixNano: "1609459214040000000" - traceId: "" + traceId: ac65000000000000c246000000000d09 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5296,9 +5413,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459214160000000" - spanId: "" + severityNumber: 9 + spanId: c9dfccf703000e09 timeUnixNano: "1609459214160000000" - traceId: "" + traceId: ad65000000000000c346000000000e09 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5341,9 +5459,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459214280000000" - spanId: "" + severityNumber: 9 + spanId: cadfccf703000f09 timeUnixNano: "1609459214280000000" - traceId: "" + traceId: ae65000000000000c446000000000f09 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5386,9 +5505,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459214400000000" - spanId: "" + severityNumber: 9 + spanId: cbdfccf703001009 timeUnixNano: "1609459214400000000" - traceId: "" + traceId: af65000000000000c546000000001009 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5431,9 +5551,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459214520000000" - spanId: "" + severityNumber: 9 + spanId: ccdfccf703001109 timeUnixNano: "1609459214520000000" - traceId: "" + traceId: b065000000000000c646000000001109 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5476,9 +5597,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459214640000000" - spanId: "" + severityNumber: 9 + spanId: cddfccf703001209 timeUnixNano: "1609459214640000000" - traceId: "" + traceId: b165000000000000c746000000001209 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: @@ -5521,9 +5643,10 @@ resourceLogs: stringValue: PUT /api/v1/series 200 250 - 0.000 ms flags: 1 observedTimeUnixNano: "1609459214760000000" - spanId: "" + severityNumber: 9 + spanId: cedfccf703001309 timeUnixNano: "1609459214760000000" - traceId: "" + traceId: b265000000000000c846000000001309 schemaUrl: https://opentelemetry.io/schemas/1.21.0 scope: attributes: diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index 365f3b77..1046625f 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -36,7 +36,7 @@ func TestMain(m *testing.M) { func setupDB( ctx context.Context, t *testing.T, - set lokie2e.BatchSet, + set *lokie2e.BatchSet, inserter logstorage.Inserter, querier logstorage.Querier, engineQuerier logqlengine.Querier, @@ -144,6 +144,15 @@ func runTest( entries int }{ // Label matchers. + // By trace id. + {`{trace_id="af36000000000000c517000000000003"}`, 1}, + {`{trace_id="AF36000000000000C517000000000003"}`, 1}, + {`{trace_id="badbadbadbadbadbaddeadbeafbadbad"}`, 0}, + // By severity. + {`{level="Info"}`, 121}, + {`{level="INFO"}`, 121}, + // All by service name. + {`{service_name="testService"}`, len(set.Records)}, // Effectively match GET. {`{http_method="GET"}`, 21}, {`{http_method=~".*GET.*"}`, 21}, @@ -177,18 +186,20 @@ func runTest( // Line filter. {`{http_method=~".+"} |= "GET"`, 21}, {`{http_method=~".+"} |= "DELETE"`, 20}, - {`{http_method=~".+"} |= "HEAD" |= "500"`, 2}, + {`{http_method=~".+"} |= "HEAD" |= " 500 "`, 2}, {`{http_method=~".+"} |~ "DELETE"`, 20}, - {`{http_method=~".+"} |~ "HEAD" |= "500"`, 2}, + {`{http_method=~".+"} |~ "HEAD" |= " 500 "`, 2}, // Try to not use offloading. {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "DELETE"`, 20}, - {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "HEAD" |= "500"`, 2}, - {`{http_method=~".+"} |= "HEAD" | line_format "{{ __line__ }}" |= "500"`, 2}, + {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "HEAD" |= " 500 "`, 2}, + {`{http_method=~".+"} |= "HEAD" | line_format "{{ __line__ }}" |= " 500 "`, 2}, // Negative line matcher. {`{http_method=~".+"} != "HEAD"`, len(set.Records) - 22}, {`{http_method=~".+"} !~ "HEAD"`, len(set.Records) - 22}, // IP line filter. {`{http_method="HEAD"} |= ip("236.7.233.166")`, 1}, + // Trace to logs. + {`{http_method=~".+"} |= "af36000000000000c517000000000003"`, 1}, // Label filter. {`{http_method=~".+"} | http_method = "GET"`, 21}, diff --git a/integration/lokie2e/generate_test.go b/integration/lokie2e/generate_test.go index b414ff6f..200f0962 100644 --- a/integration/lokie2e/generate_test.go +++ b/integration/lokie2e/generate_test.go @@ -2,6 +2,7 @@ package lokie2e_test import ( "bytes" + "encoding/binary" "fmt" "testing" "time" @@ -15,6 +16,7 @@ import ( "sigs.k8s.io/yaml" "github.com/go-faster/oteldb/integration/lokie2e" + "github.com/go-faster/oteldb/internal/otelstorage" ) func appendAttributes(target pcommon.Map, attrs []attribute.KeyValue) { @@ -34,6 +36,7 @@ func appendAttributes(target pcommon.Map, attrs []attribute.KeyValue) { } type httpLog struct { + Severity plog.SeverityNumber Time time.Time Method string Status int @@ -43,6 +46,8 @@ type httpLog struct { URL string IP string Ref string + SpanID otelstorage.SpanID + TraceID otelstorage.TraceID } func (l httpLog) Append(s *lokie2e.BatchSet) error { @@ -69,6 +74,9 @@ func (l httpLog) Append(s *lokie2e.BatchSet) error { lg.Body().SetStr(fmt.Sprintf("%s %s %d %d - 0.000 ms", l.Method, l.URL, l.Status, l.Bytes)) lg.SetTimestamp(pcommon.NewTimestampFromTime(l.Time)) lg.SetObservedTimestamp(pcommon.NewTimestampFromTime(l.Time)) + lg.SetTraceID(pcommon.TraceID(l.TraceID)) + lg.SetSpanID(pcommon.SpanID(l.SpanID)) + lg.SetSeverityNumber(l.Severity) appendAttributes(lg.Attributes(), []attribute.KeyValue{ semconv.HTTPMethod(l.Method), semconv.HTTPStatusCode(l.Status), @@ -82,7 +90,7 @@ func (l httpLog) Append(s *lokie2e.BatchSet) error { return nil } -func generateLogs(now time.Time) (s lokie2e.BatchSet, err error) { +func generateLogs(now time.Time) (*lokie2e.BatchSet, error) { type httpLogBatch struct { Method string Status int @@ -91,7 +99,7 @@ func generateLogs(now time.Time) (s lokie2e.BatchSet, err error) { Protocol string } var lines []httpLog - for _, b := range []httpLogBatch{ + for j, b := range []httpLogBatch{ {Method: "GET", Status: 200, Count: 11, IP: "200.1.1.1", Protocol: "HTTP/1.0"}, {Method: "GET", Status: 200, Count: 10, IP: "200.1.1.1", Protocol: "HTTP/1.1"}, {Method: "DELETE", Status: 200, Count: 20, IP: "200.1.1.1", Protocol: "HTTP/2.0"}, @@ -104,8 +112,36 @@ func generateLogs(now time.Time) (s lokie2e.BatchSet, err error) { {Method: "PUT", Status: 200, Count: 20, IP: "200.1.1.1", Protocol: "HTTP/2.0"}, } { for i := 0; i < b.Count; i++ { + var ( + spanID otelstorage.SpanID + traceID otelstorage.TraceID + ) + { + // Predictable IDs for testing. + binary.PutUvarint(spanID[:], uint64(i+1056123959+j*100)) + spanID[7] = byte(j) + spanID[6] = byte(i) + binary.PutUvarint(traceID[:], uint64(i+3959+j*1000)) + binary.PutUvarint(traceID[8:], uint64(i+13+j*1000)) + traceID[15] = byte(j) + traceID[14] = byte(i) + } now = now.Add(time.Millisecond * 120) + severity := plog.SeverityNumberInfo + switch b.Status / 100 { + case 2: + severity = plog.SeverityNumberInfo + case 3: + severity = plog.SeverityNumberWarn + case 4: + severity = plog.SeverityNumberError + case 5: + severity = plog.SeverityNumberFatal + } lines = append(lines, httpLog{ + Severity: severity, + SpanID: spanID, + TraceID: traceID, Time: now, Method: b.Method, Status: b.Status, @@ -118,11 +154,13 @@ func generateLogs(now time.Time) (s lokie2e.BatchSet, err error) { } } + s := lokie2e.NewBatchSet() for _, l := range lines { - if err := l.Append(&s); err != nil { + if err := l.Append(s); err != nil { return s, err } } + return s, nil } diff --git a/integration/lokie2e/loki_e2e.go b/integration/lokie2e/loki_e2e.go index f24ea6d3..81801a8b 100644 --- a/integration/lokie2e/loki_e2e.go +++ b/integration/lokie2e/loki_e2e.go @@ -20,6 +20,40 @@ type BatchSet struct { End otelstorage.Timestamp } +func NewBatchSet() *BatchSet { + s := &BatchSet{} + // Init common labels. + // Should return blank values. + s.Labels = map[string][]logstorage.Label{} + for _, v := range []string{ + logstorage.LabelBody, + logstorage.LabelTraceID, + logstorage.LabelSpanID, + logstorage.LabelServiceInstanceID, + logstorage.LabelServiceName, + logstorage.LabelServiceNamespace, + } { + s.Labels[v] = []logstorage.Label{} + } + for _, i := range []plog.SeverityNumber{ + plog.SeverityNumberUnspecified, + plog.SeverityNumberTrace, + plog.SeverityNumberDebug, + plog.SeverityNumberInfo, + plog.SeverityNumberWarn, + plog.SeverityNumberError, + plog.SeverityNumberFatal, + } { + s.addLabel(logstorage.Label{ + Name: logstorage.LabelSeverity, + Value: i.String(), + Type: int32(pcommon.ValueTypeStr), + }) + } + + return s +} + func (s *BatchSet) Append(raw plog.Logs) error { s.Batches = append(s.Batches, raw) diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index 5ea38d33..e21587ac 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -2,7 +2,9 @@ package chstorage import ( "context" + "encoding/hex" "fmt" + "slices" "strings" "github.com/ClickHouse/ch-go" @@ -11,6 +13,7 @@ import ( "github.com/go-faster/jx" "github.com/go-faster/sdk/zctx" "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/plog" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" @@ -70,6 +73,28 @@ LIMIT 1000`, return nil, errors.Wrap(err, "select") } + // Append materialized labels. + out = append(out, + logstorage.LabelTraceID, + logstorage.LabelSpanID, + logstorage.LabelSeverity, + logstorage.LabelBody, + logstorage.LabelServiceName, + logstorage.LabelServiceInstanceID, + logstorage.LabelServiceNamespace, + ) + + // Deduplicate. + seen := make(map[string]struct{}, len(out)) + for _, v := range out { + seen[v] = struct{}{} + } + out = out[:0] + for k := range seen { + out = append(out, k) + } + slices.Sort(out) + return out, nil } @@ -168,6 +193,26 @@ func (q *Querier) LabelValues(ctx context.Context, labelName string, opts logsto } span.End() }() + switch labelName { + case logstorage.LabelBody, logstorage.LabelSpanID, logstorage.LabelTraceID: + return &labelStaticIterator{ + name: labelName, + values: nil, + }, nil + case logstorage.LabelSeverity: + return &labelStaticIterator{ + name: labelName, + values: []jx.Raw{ + jx.Raw(plog.SeverityNumberUnspecified.String()), + jx.Raw(plog.SeverityNumberTrace.String()), + jx.Raw(plog.SeverityNumberDebug.String()), + jx.Raw(plog.SeverityNumberInfo.String()), + jx.Raw(plog.SeverityNumberWarn.String()), + jx.Raw(plog.SeverityNumberError.String()), + jx.Raw(plog.SeverityNumberFatal.String()), + }, + }, nil + } { mapping, err := q.getLabelMapping(ctx, []string{labelName}) if err != nil { @@ -290,30 +335,82 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta default: return nil, errors.Errorf("unexpected op %q", m.Op) } - for i, column := range []string{ - "attributes", - "resource", - "scope_attributes", - } { - if i != 0 { - query.WriteString(" OR ") + switch labelName { + case logstorage.LabelTraceID: + switch m.Op { + case logql.OpEq, logql.OpNotEq: + fmt.Fprintf(&query, "trace_id = unhex(%s)", singleQuoted(m.Value)) + case logql.OpRe, logql.OpNotRe: + fmt.Fprintf(&query, "hex(trace_id) REGEXP %s", singleQuoted(m.Value)) + } + case logstorage.LabelSpanID: + switch m.Op { + case logql.OpEq, logql.OpNotEq: + fmt.Fprintf(&query, "span_id = unhex(%s)", singleQuoted(m.Value)) + case logql.OpRe, logql.OpNotRe: + fmt.Fprintf(&query, "hex(span_id) REGEXP %s", singleQuoted(m.Value)) + } + case logstorage.LabelSeverity: + switch m.Op { + case logql.OpEq, logql.OpNotEq: + // Direct comparison with severity number. + var severityNumber uint8 + for i := plog.SeverityNumberUnspecified; i <= plog.SeverityNumberFatal4; i++ { + if strings.ToLower(i.String()) == strings.ToLower(m.Value) { + severityNumber = uint8(i) + break + } + } + fmt.Fprintf(&query, "severity_number = %d", severityNumber) + default: + // TODO(ernado): just do regex in-place and add `IN (...)` to query. + return nil, errors.Errorf("%q not implemented for severity", m.Op) + } + case logstorage.LabelBody: + switch m.Op { + case logql.OpEq, logql.OpNotEq: + fmt.Fprintf(&query, "positionUTF8(body, %s) > 0", singleQuoted(m.Value)) + case logql.OpRe, logql.OpNotRe: + fmt.Fprintf(&query, "service_name REGEXP %s", singleQuoted(m.Value)) } - // TODO: how to match integers, booleans, floats, arrays? + case logstorage.LabelServiceName, logstorage.LabelServiceNamespace, logstorage.LabelServiceInstanceID: + // Materialized from resource.service.{name,namespace,instance_id}. switch m.Op { case logql.OpEq, logql.OpNotEq: - fmt.Fprintf(&query, "JSONExtractString(%s, %s) = %s", column, singleQuoted(labelName), singleQuoted(m.Value)) + fmt.Fprintf(&query, "positionUTF8(%s, %s) > 0", labelName, singleQuoted(m.Value)) case logql.OpRe, logql.OpNotRe: - fmt.Fprintf(&query, "JSONExtractString(%s, %s) REGEXP %s", column, singleQuoted(labelName), singleQuoted(m.Value)) + fmt.Fprintf(&query, "%s REGEXP %s", labelName, singleQuoted(m.Value)) + } + default: + // Search in all attributes. + for i, column := range []string{ + "attributes", + "resource", + "scope_attributes", + } { + if i != 0 { + query.WriteString(" OR ") + } + // TODO: how to match integers, booleans, floats, arrays? + switch m.Op { + case logql.OpEq, logql.OpNotEq: + fmt.Fprintf(&query, "JSONExtractString(%s, %s) = %s", column, singleQuoted(labelName), singleQuoted(m.Value)) + case logql.OpRe, logql.OpNotRe: + fmt.Fprintf(&query, "JSONExtractString(%s, %s) REGEXP %s", column, singleQuoted(labelName), singleQuoted(m.Value)) + } } } query.WriteByte(')') } + + hasTraceID := false + for _, m := range params.Line { switch m.Op { case logql.OpEq, logql.OpRe: - query.WriteString(" AND ") + query.WriteString(" AND (") case logql.OpNotEq, logql.OpNotRe: - query.WriteString(" AND NOT ") + query.WriteString(" AND NOT (") default: return nil, errors.Errorf("unexpected op %q", m.Op) } @@ -321,12 +418,30 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta switch m.Op { case logql.OpEq, logql.OpNotEq: fmt.Fprintf(&query, "positionUTF8(body, %s) > 0", singleQuoted(m.Value)) + { + // HACK: check for special case of hex-encoded trace_id and span_id. + // Like `{http_method=~".+"} |= "af36000000000000c517000000000003"`. + // TODO(ernado): also handle regex? + encoded := strings.ToLower(m.Value) + v, _ := hex.DecodeString(encoded) + switch len(v) { + case len(otelstorage.TraceID{}): + fmt.Fprintf(&query, " OR trace_id = unhex(%s)", singleQuoted(encoded)) + hasTraceID = true + case len(otelstorage.SpanID{}): + fmt.Fprintf(&query, " OR span_id = unhex(%s)", singleQuoted(encoded)) + } + } case logql.OpRe, logql.OpNotRe: fmt.Fprintf(&query, "body REGEXP %s", singleQuoted(m.Value)) } + query.WriteByte(')') } // TODO: use streaming. + if hasTraceID { + fmt.Println(query.String()) + } var data []logstorage.Record if err := q.ch.Do(ctx, ch.Query{ Logger: zctx.From(ctx).Named("ch"), @@ -343,5 +458,6 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta }); err != nil { return nil, errors.Wrap(err, "select") } + fmt.Println("got data", len(data)) return &logStaticIterator{data: data}, nil } diff --git a/internal/logql/logqlengine/label_set.go b/internal/logql/logqlengine/label_set.go index 609948c3..aa1572a2 100644 --- a/internal/logql/logqlengine/label_set.go +++ b/internal/logql/logqlengine/label_set.go @@ -80,15 +80,15 @@ func (l *LabelSet) SetFromRecord(record logstorage.Record) { l.reset() if traceID := record.TraceID; !traceID.IsEmpty() { - l.Set(`trace_id`, pcommon.NewValueStr(traceID.Hex())) + l.Set(logstorage.LabelTraceID, pcommon.NewValueStr(traceID.Hex())) } if spanID := record.SpanID; !spanID.IsEmpty() { - l.Set(`span_id`, pcommon.NewValueStr(spanID.Hex())) + l.Set(logstorage.LabelSpanID, pcommon.NewValueStr(spanID.Hex())) } if severity := record.SeverityNumber; severity != plog.SeverityNumberUnspecified { - l.Set(`level`, pcommon.NewValueStr(severity.String())) + l.Set(logstorage.LabelSeverity, pcommon.NewValueStr(severity.String())) } - l.Set(`msg`, pcommon.NewValueStr(record.Body)) + l.Set(logstorage.LabelBody, pcommon.NewValueStr(record.Body)) l.SetAttrs(record.Attrs, record.ScopeAttrs, record.ResourceAttrs) } diff --git a/internal/logql/logqlengine/otel_adapter.go b/internal/logql/logqlengine/otel_adapter.go index 6c06669b..aa94f9ae 100644 --- a/internal/logql/logqlengine/otel_adapter.go +++ b/internal/logql/logqlengine/otel_adapter.go @@ -12,9 +12,23 @@ func LineFromRecord(record logstorage.Record) string { // Create JSON object from record. e := &jx.Encoder{} e.Obj(func(e *jx.Encoder) { - e.Field("msg", func(e *jx.Encoder) { + e.Field(logstorage.LabelBody, func(e *jx.Encoder) { e.Str(record.Body) }) + + // HACK: add trace_id, span_id so "trace to logs" metrics work. + // Like `{http_method=~".+"} |= "af36000000000000c517000000000003"`. + if !record.TraceID.IsEmpty() { + e.Field(logstorage.LabelTraceID, func(e *jx.Encoder) { + e.Str(record.TraceID.Hex()) + }) + } + if !record.SpanID.IsEmpty() { + e.Field(logstorage.LabelSpanID, func(e *jx.Encoder) { + e.Str(record.SpanID.Hex()) + }) + } + record.Attrs.AsMap().Range(func(k string, v pcommon.Value) bool { e.Field(k, func(e *jx.Encoder) { switch v.Type() { diff --git a/internal/logstorage/schema.go b/internal/logstorage/schema.go index a3a1fbb1..1522de9e 100644 --- a/internal/logstorage/schema.go +++ b/internal/logstorage/schema.go @@ -7,6 +7,18 @@ import ( "github.com/go-faster/oteldb/internal/otelstorage" ) +// Labels to use where prometheus compatible labels are required, e.g. loki. +const ( + LabelTraceID = "trace_id" + LabelSpanID = "span_id" + LabelSeverity = "level" + LabelBody = "msg" + + LabelServiceName = "service_name" // resource.service.name + LabelServiceNamespace = "service_namespace" // resource.service.namespace + LabelServiceInstanceID = "service_instance_id" // resource.service.instance_id +) + // Record is a log record. type Record struct { Timestamp otelstorage.Timestamp `json:"timestamp" yson:"timestamp"` diff --git a/internal/otelschema/_golden/ddl.sql b/internal/otelschema/_golden/ddl.sql index ff81dece..a9c5e513 100644 --- a/internal/otelschema/_golden/ddl.sql +++ b/internal/otelschema/_golden/ddl.sql @@ -85,6 +85,7 @@ CREATE TABLE columns ( db_elasticsearch_cluster_name String COMMENT 'db.elasticsearch.cluster.name', db_elasticsearch_node_name String COMMENT 'db.elasticsearch.node.name', db_elasticsearch_path_parts String COMMENT 'db.elasticsearch.path_parts', + db_instance_id String COMMENT 'db.instance.id', db_jdbc_driver_classname String COMMENT 'db.jdbc.driver_classname', db_mongodb_collection String COMMENT 'db.mongodb.collection', db_mssql_instance_name String COMMENT 'db.mssql.instance_name', @@ -219,6 +220,7 @@ CREATE TABLE columns ( k8s_namespace_name String COMMENT 'k8s.namespace.name', k8s_node_name String COMMENT 'k8s.node.name', k8s_node_uid UUID COMMENT 'k8s.node.uid', + k8s_pod_labels String COMMENT 'k8s.pod.labels', k8s_pod_name String COMMENT 'k8s.pod.name', k8s_pod_uid UUID COMMENT 'k8s.pod.uid', k8s_replicaset_name String COMMENT 'k8s.replicaset.name', diff --git a/internal/otelschema/_golden/registry.yaml b/internal/otelschema/_golden/registry.yaml index 22323127..b1cbd77d 100644 --- a/internal/otelschema/_golden/registry.yaml +++ b/internal/otelschema/_golden/registry.yaml @@ -623,6 +623,15 @@ entries: group: db name: db_elasticsearch_path_parts type: template[string] + db.instance.id: + brief: | + An identifier (address, unique name, or any other identifier) of the database instance that is executing queries or mutations on the current connection. This is useful in cases where the database is running in a clustered environment and the instrumentation is able to record the node executing the query. The client may obtain this value in databases like MySQL using queries like `select @@hostname`. + column: String + examples: + - mysql-e26b99z.example.com + group: db + name: db_instance_id + type: string db.jdbc.driver_classname: brief: | The fully-qualified class name of the [Java Database Connectivity (JDBC)](https://docs.oracle.com/javase/8/docs/technotes/guides/jdbc/) driver used to connect. @@ -1561,6 +1570,17 @@ entries: group: k8s name: k8s_node_uid type: string + k8s.pod.labels: + brief: | + The labels placed on the Pod, the `` being the label name, the value being the label value. + column: String + examples: + - k8s.pod.labels.app=my-app + - k8s.pod.labels.mycompany.io/arch=x64 + - k8s.pod.labels.data= + group: k8s + name: k8s_pod_labels + type: template[string] k8s.pod.name: brief: | The name of the Pod. @@ -3100,5 +3120,5 @@ entries: statistics: deprecated: 24 enum: 23 - total: 334 + total: 336 unknown: 0 From ff6ebe9c1117e61738b5fd9920735560f21c2075 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 19:24:11 +0300 Subject: [PATCH 100/112] feat(logqlengine): move trace_id, span_id to the tail of json --- internal/logql/logqlengine/otel_adapter.go | 26 ++++++++++------------ 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/internal/logql/logqlengine/otel_adapter.go b/internal/logql/logqlengine/otel_adapter.go index aa94f9ae..8d106644 100644 --- a/internal/logql/logqlengine/otel_adapter.go +++ b/internal/logql/logqlengine/otel_adapter.go @@ -15,20 +15,6 @@ func LineFromRecord(record logstorage.Record) string { e.Field(logstorage.LabelBody, func(e *jx.Encoder) { e.Str(record.Body) }) - - // HACK: add trace_id, span_id so "trace to logs" metrics work. - // Like `{http_method=~".+"} |= "af36000000000000c517000000000003"`. - if !record.TraceID.IsEmpty() { - e.Field(logstorage.LabelTraceID, func(e *jx.Encoder) { - e.Str(record.TraceID.Hex()) - }) - } - if !record.SpanID.IsEmpty() { - e.Field(logstorage.LabelSpanID, func(e *jx.Encoder) { - e.Str(record.SpanID.Hex()) - }) - } - record.Attrs.AsMap().Range(func(k string, v pcommon.Value) bool { e.Field(k, func(e *jx.Encoder) { switch v.Type() { @@ -47,6 +33,18 @@ func LineFromRecord(record logstorage.Record) string { }) return true }) + // HACK: add trace_id, span_id so "trace to logs" metrics work. + // Like `{http_method=~".+"} |= "af36000000000000c517000000000003"`. + if !record.TraceID.IsEmpty() { + e.Field(logstorage.LabelTraceID, func(e *jx.Encoder) { + e.Str(record.TraceID.Hex()) + }) + } + if !record.SpanID.IsEmpty() { + e.Field(logstorage.LabelSpanID, func(e *jx.Encoder) { + e.Str(record.SpanID.Hex()) + }) + } }) return e.String() } From 51f2035a6a4f85f0c5f6764eb4d286260ecb0f68 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 20:16:47 +0300 Subject: [PATCH 101/112] fix(logqlengine): fix NPE --- internal/logql/logqlengine/otel_adapter.go | 37 ++++++++++++---------- 1 file changed, 20 insertions(+), 17 deletions(-) diff --git a/internal/logql/logqlengine/otel_adapter.go b/internal/logql/logqlengine/otel_adapter.go index 8d106644..e5434d51 100644 --- a/internal/logql/logqlengine/otel_adapter.go +++ b/internal/logql/logqlengine/otel_adapter.go @@ -15,24 +15,27 @@ func LineFromRecord(record logstorage.Record) string { e.Field(logstorage.LabelBody, func(e *jx.Encoder) { e.Str(record.Body) }) - record.Attrs.AsMap().Range(func(k string, v pcommon.Value) bool { - e.Field(k, func(e *jx.Encoder) { - switch v.Type() { - case pcommon.ValueTypeStr: - e.Str(v.Str()) - case pcommon.ValueTypeBool: - e.Bool(v.Bool()) - case pcommon.ValueTypeInt: - e.Int64(v.Int()) - case pcommon.ValueTypeDouble: - e.Float64(v.Double()) - default: - // Fallback. - e.Str(v.AsString()) - } + + if m := record.Attrs.AsMap(); m != (pcommon.Map{}) { + record.Attrs.AsMap().Range(func(k string, v pcommon.Value) bool { + e.Field(k, func(e *jx.Encoder) { + switch v.Type() { + case pcommon.ValueTypeStr: + e.Str(v.Str()) + case pcommon.ValueTypeBool: + e.Bool(v.Bool()) + case pcommon.ValueTypeInt: + e.Int64(v.Int()) + case pcommon.ValueTypeDouble: + e.Float64(v.Double()) + default: + // Fallback. + e.Str(v.AsString()) + } + }) + return true }) - return true - }) + } // HACK: add trace_id, span_id so "trace to logs" metrics work. // Like `{http_method=~".+"} |= "af36000000000000c517000000000003"`. if !record.TraceID.IsEmpty() { From 7bd445969044c86cdac2668074ed31703d3278b0 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 20:18:17 +0300 Subject: [PATCH 102/112] feat(zapotel): init zap to otel --- go.mod | 30 +++---- internal/zapotel/zapotel.go | 153 ++++++++++++++++++++++++++++++++++++ 2 files changed, 168 insertions(+), 15 deletions(-) create mode 100644 internal/zapotel/zapotel.go diff --git a/go.mod b/go.mod index 2703a269..5a86b530 100644 --- a/go.mod +++ b/go.mod @@ -12,7 +12,7 @@ require ( github.com/fatih/color v1.16.0 github.com/go-faster/errors v0.7.0 github.com/go-faster/jx v1.1.0 - github.com/go-faster/sdk v0.10.2 + github.com/go-faster/sdk v0.11.0 github.com/go-faster/tcpproxy v0.1.0 github.com/go-logfmt/logfmt v0.6.0 github.com/google/uuid v1.4.0 @@ -40,8 +40,8 @@ require ( go.opentelemetry.io/contrib/zpages v0.46.1 go.opentelemetry.io/otel v1.21.0 go.opentelemetry.io/otel/bridge/opentracing v1.21.0 - go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.20.0 - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.20.0 + go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0 + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0 go.opentelemetry.io/otel/metric v1.21.0 go.opentelemetry.io/otel/sdk v1.21.0 go.opentelemetry.io/otel/sdk/metric v1.21.0 @@ -178,19 +178,19 @@ require ( go.opentelemetry.io/collector/processor v0.89.0 // indirect go.opentelemetry.io/collector/semconv v0.89.0 // indirect go.opentelemetry.io/collector/service v0.89.0 // indirect - go.opentelemetry.io/contrib/instrumentation/runtime v0.46.0 // indirect - go.opentelemetry.io/contrib/propagators/autoprop v0.46.0 // indirect - go.opentelemetry.io/contrib/propagators/aws v1.21.0 // indirect - go.opentelemetry.io/contrib/propagators/b3 v1.21.0 // indirect - go.opentelemetry.io/contrib/propagators/jaeger v1.21.0 // indirect - go.opentelemetry.io/contrib/propagators/ot v1.21.0 // indirect + go.opentelemetry.io/contrib/instrumentation/runtime v0.46.1 // indirect + go.opentelemetry.io/contrib/propagators/autoprop v0.46.1 // indirect + go.opentelemetry.io/contrib/propagators/aws v1.21.1 // indirect + go.opentelemetry.io/contrib/propagators/b3 v1.21.1 // indirect + go.opentelemetry.io/contrib/propagators/jaeger v1.21.1 // indirect + go.opentelemetry.io/contrib/propagators/ot v1.21.1 // indirect go.opentelemetry.io/otel/bridge/opencensus v0.43.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v0.43.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.43.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.20.0 // indirect - go.opentelemetry.io/otel/exporters/prometheus v0.43.0 // indirect - go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v0.43.0 // indirect - go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.20.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v0.44.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.44.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0 // indirect + go.opentelemetry.io/otel/exporters/prometheus v0.44.0 // indirect + go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v0.44.0 // indirect + go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.21.0 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/automaxprocs v1.5.3 // indirect go.uber.org/goleak v1.3.0 // indirect diff --git a/internal/zapotel/zapotel.go b/internal/zapotel/zapotel.go new file mode 100644 index 00000000..59465eb4 --- /dev/null +++ b/internal/zapotel/zapotel.go @@ -0,0 +1,153 @@ +// Package zapotel provides OpenTelemetry logs exporter zap core implementation. +package zapotel + +import ( + "context" + "encoding/hex" + "math" + "strings" + + "github.com/go-faster/errors" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/plog" + "go.opentelemetry.io/collector/pdata/plog/plogotlp" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/sdk/resource" + "go.uber.org/zap/zapcore" +) + +// New initializes new zapcore.Core from grpc client and resource. +func New(res *resource.Resource, client plogotlp.GRPCClient) zapcore.Core { + return &contextObserver{ + client: client, + res: res, + } +} + +type contextObserver struct { + zapcore.LevelEnabler + context []zapcore.Field + client plogotlp.GRPCClient + res *resource.Resource +} + +var ( + _ zapcore.Core = (*contextObserver)(nil) + _ zapcore.LevelEnabler = (*contextObserver)(nil) +) + +func (co *contextObserver) Level() zapcore.Level { + return zapcore.LevelOf(co.LevelEnabler) +} + +func (co *contextObserver) Check(ent zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { + if co.Enabled(ent.Level) { + return ce.AddCore(ent, co) + } + return ce +} + +func (co *contextObserver) With(fields []zapcore.Field) zapcore.Core { + return &contextObserver{ + LevelEnabler: co.LevelEnabler, + context: append(co.context[:len(co.context):len(co.context)], fields...), + } +} + +func (co *contextObserver) toLogs(ent zapcore.Entry, fields []zapcore.Field) plog.Logs { + var ( + ld = plog.NewLogs() + rl = ld.ResourceLogs().AppendEmpty() + ) + { + a := rl.Resource().Attributes() + for _, kv := range co.res.Attributes() { + k := string(kv.Key) + switch kv.Value.Type() { + case attribute.STRING: + a.PutStr(k, kv.Value.AsString()) + case attribute.BOOL: + a.PutBool(k, kv.Value.AsBool()) + default: + a.PutStr(k, kv.Value.AsString()) + } + } + } + + il := rl.ScopeLogs().AppendEmpty() + + scope := il.Scope() + scope.SetName("zapotel") + scope.SetVersion("v0.1") + + lg := il.LogRecords().AppendEmpty() + lg.Body().SetStr(ent.Message) + lg.SetTimestamp(pcommon.NewTimestampFromTime(ent.Time)) + lg.SetObservedTimestamp(pcommon.NewTimestampFromTime(ent.Time)) + { + a := lg.Attributes() + var skipped uint32 + for _, f := range fields { + k := f.Key + switch f.Type { + case zapcore.BoolType: + a.PutBool(k, f.Integer == 1) + case zapcore.StringType: + l := len(f.String) + if (k == "trace_id" && l == 32) || (k == "span_id" && l == 16) { + // Checking for tracing. + var ( + traceID pcommon.TraceID + spanID pcommon.SpanID + ) + v, err := hex.DecodeString(strings.ToLower(f.String)) + if err == nil { + switch k { + case "trace_id": + copy(traceID[:], v) + lg.SetTraceID(traceID) + case "span_id": + copy(spanID[:], v) + lg.SetSpanID(spanID) + } + // Don't add as regular string. + continue + } + } + a.PutStr(k, f.String) + case zapcore.Int8Type, zapcore.Int16Type, zapcore.Int32Type, zapcore.Int64Type, + zapcore.Uint8Type, zapcore.Uint16Type, zapcore.Uint32Type, zapcore.Uint64Type: + a.PutInt(k, f.Integer) + case zapcore.Float32Type: + a.PutDouble(k, float64(math.Float32frombits(uint32(f.Integer)))) + case zapcore.Float64Type: + a.PutDouble(k, math.Float64frombits(uint64(f.Integer))) + default: + // Time, duration, "any", ... + // TODO(ernado): support + skipped++ + } + } + if skipped > 0 { + scope.SetDroppedAttributesCount(skipped) + } + } + return ld +} + +func (co *contextObserver) Write(ent zapcore.Entry, fields []zapcore.Field) error { + all := make([]zapcore.Field, 0, len(fields)+len(co.context)) + all = append(all, co.context...) + all = append(all, fields...) + + ctx := context.TODO() + + logs := co.toLogs(ent, all) + if _, err := co.client.Export(ctx, plogotlp.NewExportRequestFromLogs(logs)); err != nil { + return errors.Wrap(err, "send logs") + } + + return nil +} + +func (co *contextObserver) Sync() error { return nil } From 484200777ba776c0c4dcf7793796936dea4ec220 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 20:50:20 +0300 Subject: [PATCH 103/112] feat: enable zapotel --- cmd/oteldb/oteldb.go | 41 +++++++++++++++++++ dev/local/ch/docker-compose.yml | 3 ++ internal/zapotel/zapotel.go | 71 +++++++++++++++++++++------------ 3 files changed, 90 insertions(+), 25 deletions(-) diff --git a/cmd/oteldb/oteldb.go b/cmd/oteldb/oteldb.go index 6416e64c..497c6ca7 100644 --- a/cmd/oteldb/oteldb.go +++ b/cmd/oteldb/oteldb.go @@ -2,15 +2,24 @@ package main import ( "context" + "os" + "strings" "github.com/go-faster/errors" "github.com/go-faster/sdk/app" + "github.com/go-faster/sdk/zctx" "github.com/opentracing/opentracing-go" + "go.opentelemetry.io/collector/pdata/plog/plogotlp" + "go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc" "go.opentelemetry.io/otel" otelBridge "go.opentelemetry.io/otel/bridge/opentracing" "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" "github.com/go-faster/oteldb/internal/autozpages" + "github.com/go-faster/oteldb/internal/zapotel" ) func main() { @@ -34,6 +43,38 @@ func main() { // Override for context propagation. otel.SetTracerProvider(wrapperTracerProvider) m = m.WithTracerProvider(wrapperTracerProvider) + + } + if os.Getenv("OTEL_LOGS_EXPORTER") == "otlp" { + // Setting zap -> otel. + otelOptions := []otelgrpc.Option{ + otelgrpc.WithTracerProvider(m.TracerProvider()), + otelgrpc.WithMeterProvider(m.MeterProvider()), + } + // Only PoC, should be replaced with real initialization + // and moved to go-faster/sdk. + endpoint := os.Getenv("OTEL_EXPORTER_OTLP_LOGS_ENDPOINT") + endpoint = strings.TrimPrefix(endpoint, "http://") + conn, err := grpc.DialContext(ctx, endpoint, + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithStatsHandler(otelgrpc.NewClientHandler(otelOptions...)), + ) + if err != nil { + return errors.Wrap(err, "dial logs endpoint") + } + res, err := app.Resource(ctx) + if err != nil { + return errors.Wrap(err, "get resource") + } + otelCore := zapotel.New(zap.InfoLevel, res, plogotlp.NewGRPCClient(conn)) + // Update logger down the stack. + lg.Info("Setting up OTLP log exporter") + lg = lg.WithOptions( + zap.WrapCore(func(core zapcore.Core) zapcore.Core { + return zapcore.NewTee(core, otelCore) + }), + ) + ctx = zctx.Base(ctx, lg) } root, err := newApp(ctx, lg, m) diff --git a/dev/local/ch/docker-compose.yml b/dev/local/ch/docker-compose.yml index 602e0fe6..49d7ebcd 100644 --- a/dev/local/ch/docker-compose.yml +++ b/dev/local/ch/docker-compose.yml @@ -70,6 +70,9 @@ services: - OTEL_METRICS_EXPORTER=prometheus - OTEL_EXPORTER_OTLP_TRACES_PROTOCOL=grpc - OTEL_EXPORTER_OTLP_TRACES_ENDPOINT=http://otelcol:4317 + - OTEL_LOGS_EXPORTER=otlp + - OTEL_EXPORTER_OTLP_LOGS_PROTOCOL=grpc + - OTEL_EXPORTER_OTLP_LOGS_ENDPOINT=http://otelcol:4317 - OTEL_RESOURCE_ATTRIBUTES=service.name=go-faster.oteldb ports: # Querier ports diff --git a/internal/zapotel/zapotel.go b/internal/zapotel/zapotel.go index 59465eb4..61714a4a 100644 --- a/internal/zapotel/zapotel.go +++ b/internal/zapotel/zapotel.go @@ -17,14 +17,15 @@ import ( ) // New initializes new zapcore.Core from grpc client and resource. -func New(res *resource.Resource, client plogotlp.GRPCClient) zapcore.Core { - return &contextObserver{ - client: client, - res: res, +func New(enab zapcore.LevelEnabler, res *resource.Resource, client plogotlp.GRPCClient) zapcore.Core { + return &exporter{ + LevelEnabler: enab, + client: client, + res: res, } } -type contextObserver struct { +type exporter struct { zapcore.LevelEnabler context []zapcore.Field client plogotlp.GRPCClient @@ -32,36 +33,38 @@ type contextObserver struct { } var ( - _ zapcore.Core = (*contextObserver)(nil) - _ zapcore.LevelEnabler = (*contextObserver)(nil) + _ zapcore.Core = (*exporter)(nil) + _ zapcore.LevelEnabler = (*exporter)(nil) ) -func (co *contextObserver) Level() zapcore.Level { - return zapcore.LevelOf(co.LevelEnabler) +func (e *exporter) Level() zapcore.Level { + return zapcore.LevelOf(e.LevelEnabler) } -func (co *contextObserver) Check(ent zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { - if co.Enabled(ent.Level) { - return ce.AddCore(ent, co) +func (e *exporter) Check(ent zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { + if e.Enabled(ent.Level) { + return ce.AddCore(ent, e) } return ce } -func (co *contextObserver) With(fields []zapcore.Field) zapcore.Core { - return &contextObserver{ - LevelEnabler: co.LevelEnabler, - context: append(co.context[:len(co.context):len(co.context)], fields...), +func (e *exporter) With(fields []zapcore.Field) zapcore.Core { + return &exporter{ + LevelEnabler: e.LevelEnabler, + client: e.client, + res: e.res, + context: append(e.context[:len(e.context):len(e.context)], fields...), } } -func (co *contextObserver) toLogs(ent zapcore.Entry, fields []zapcore.Field) plog.Logs { +func (e *exporter) toLogs(ent zapcore.Entry, fields []zapcore.Field) plog.Logs { var ( ld = plog.NewLogs() rl = ld.ResourceLogs().AppendEmpty() ) { a := rl.Resource().Attributes() - for _, kv := range co.res.Attributes() { + for _, kv := range e.res.Attributes() { k := string(kv.Key) switch kv.Value.Type() { case attribute.STRING: @@ -82,6 +85,24 @@ func (co *contextObserver) toLogs(ent zapcore.Entry, fields []zapcore.Field) plo lg := il.LogRecords().AppendEmpty() lg.Body().SetStr(ent.Message) + // TODO: update mapping from spec + switch ent.Level { + case zapcore.DebugLevel: + lg.SetSeverityNumber(plog.SeverityNumberDebug) + case zapcore.InfoLevel: + lg.SetSeverityNumber(plog.SeverityNumberInfo) + case zapcore.WarnLevel: + lg.SetSeverityNumber(plog.SeverityNumberWarn) + case zapcore.ErrorLevel: + lg.SetSeverityNumber(plog.SeverityNumberError) + case zapcore.DPanicLevel: + lg.SetSeverityNumber(plog.SeverityNumberFatal) + case zapcore.PanicLevel: + lg.SetSeverityNumber(plog.SeverityNumberFatal) + case zapcore.FatalLevel: + lg.SetSeverityNumber(plog.SeverityNumberFatal) + } + lg.SetSeverityText(ent.Level.String()) lg.SetTimestamp(pcommon.NewTimestampFromTime(ent.Time)) lg.SetObservedTimestamp(pcommon.NewTimestampFromTime(ent.Time)) { @@ -135,19 +156,19 @@ func (co *contextObserver) toLogs(ent zapcore.Entry, fields []zapcore.Field) plo return ld } -func (co *contextObserver) Write(ent zapcore.Entry, fields []zapcore.Field) error { - all := make([]zapcore.Field, 0, len(fields)+len(co.context)) - all = append(all, co.context...) +func (e *exporter) Write(ent zapcore.Entry, fields []zapcore.Field) error { + all := make([]zapcore.Field, 0, len(fields)+len(e.context)) + all = append(all, e.context...) all = append(all, fields...) ctx := context.TODO() - - logs := co.toLogs(ent, all) - if _, err := co.client.Export(ctx, plogotlp.NewExportRequestFromLogs(logs)); err != nil { + logs := e.toLogs(ent, all) + req := plogotlp.NewExportRequestFromLogs(logs) + if _, err := e.client.Export(ctx, req); err != nil { return errors.Wrap(err, "send logs") } return nil } -func (co *contextObserver) Sync() error { return nil } +func (e *exporter) Sync() error { return nil } From 9d0a43a14ef66f3ebeb919184c7028a8d52dc1e0 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 20:50:29 +0300 Subject: [PATCH 104/112] chore: pass ch.Query.Logger --- internal/chstorage/inserter_logs.go | 16 ++++++++++------ internal/chstorage/inserter_metrics.go | 6 ++++-- internal/chstorage/inserter_traces.go | 11 +++++++---- internal/chstorage/querier_metrics.go | 9 +++++++-- internal/chstorage/querier_traces.go | 7 +++++-- internal/chstorage/schema.go | 4 +++- 6 files changed, 36 insertions(+), 17 deletions(-) diff --git a/internal/chstorage/inserter_logs.go b/internal/chstorage/inserter_logs.go index 9cf04047..c148c081 100644 --- a/internal/chstorage/inserter_logs.go +++ b/internal/chstorage/inserter_logs.go @@ -5,6 +5,7 @@ import ( "github.com/ClickHouse/ch-go" "github.com/go-faster/errors" + "github.com/go-faster/sdk/zctx" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" @@ -38,8 +39,9 @@ func (i *Inserter) InsertLogLabels(ctx context.Context, set map[logstorage.Label attrs.AddRow(name, label.Name) } if err := i.ch.Do(ctx, ch.Query{ - Body: attrs.Input().Into(table), - Input: attrs.Input(), + Logger: zctx.From(ctx), + Body: attrs.Input().Into(table), + Input: attrs.Input(), }); err != nil { return errors.Wrap(err, "insert labels") } @@ -67,8 +69,9 @@ func (i *Inserter) InsertRecords(ctx context.Context, records []logstorage.Recor i.mapRecords(logs, records) if err := i.ch.Do(ctx, ch.Query{ - Body: logs.Input().Into(table), - Input: logs.Input(), + Logger: zctx.From(ctx), + Body: logs.Input().Into(table), + Input: logs.Input(), }); err != nil { return errors.Wrap(err, "insert records") } @@ -80,8 +83,9 @@ func (i *Inserter) InsertRecords(ctx context.Context, records []logstorage.Recor attrs.AddAttrs(record.ScopeAttrs) } if err := i.ch.Do(ctx, ch.Query{ - Body: attrs.Input().Into(i.tables.LogAttrs), - Input: attrs.Input(), + Logger: zctx.From(ctx), + Body: attrs.Input().Into(i.tables.LogAttrs), + Input: attrs.Input(), }); err != nil { return errors.Wrap(err, "insert labels") } diff --git a/internal/chstorage/inserter_metrics.go b/internal/chstorage/inserter_metrics.go index 3aded365..020ec1a1 100644 --- a/internal/chstorage/inserter_metrics.go +++ b/internal/chstorage/inserter_metrics.go @@ -9,6 +9,7 @@ import ( "github.com/ClickHouse/ch-go/chpool" "github.com/ClickHouse/ch-go/proto" "github.com/go-faster/errors" + "github.com/go-faster/sdk/zctx" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" "golang.org/x/sync/errgroup" @@ -71,8 +72,9 @@ func (b *metricsBatch) Insert(ctx context.Context, tables Tables, client *chpool input := table.columns.Input() if err := client.Do(ctx, ch.Query{ - Body: input.Into(table.name), - Input: input, + Logger: zctx.From(ctx), + Body: input.Into(table.name), + Input: input, }); err != nil { return errors.Wrapf(err, "insert %q", table.name) } diff --git a/internal/chstorage/inserter_traces.go b/internal/chstorage/inserter_traces.go index 2589c2e4..adad39fc 100644 --- a/internal/chstorage/inserter_traces.go +++ b/internal/chstorage/inserter_traces.go @@ -5,6 +5,7 @@ import ( "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/proto" + "github.com/go-faster/sdk/zctx" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" @@ -33,8 +34,9 @@ func (i *Inserter) InsertSpans(ctx context.Context, spans []tracestorage.Span) ( } input := c.Input() return i.ch.Do(ctx, ch.Query{ - Body: input.Into(table), - Input: input, + Logger: zctx.From(ctx), + Body: input.Into(table), + Input: input, }) } @@ -73,7 +75,8 @@ func (i *Inserter) InsertTags(ctx context.Context, tags map[tracestorage.Tag]str } return i.ch.Do(ctx, ch.Query{ - Body: input.Into(table), - Input: input, + Logger: zctx.From(ctx), + Body: input.Into(table), + Input: input, }) } diff --git a/internal/chstorage/querier_metrics.go b/internal/chstorage/querier_metrics.go index 17d32226..3c50dfb7 100644 --- a/internal/chstorage/querier_metrics.go +++ b/internal/chstorage/querier_metrics.go @@ -9,6 +9,7 @@ import ( "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/proto" "github.com/go-faster/errors" + "github.com/go-faster/sdk/zctx" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" @@ -80,7 +81,8 @@ func (p *promQuerier) LabelValues(ctx context.Context, name string, matchers ... var column proto.ColStr if err := p.ch.Do(ctx, ch.Query{ - Body: query.String(), + Logger: zctx.From(ctx), + Body: query.String(), Result: proto.Results{ {Name: "value", Data: &column}, }, @@ -123,7 +125,8 @@ func (p *promQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matche column := new(proto.ColStr).LowCardinality() if err := p.ch.Do(ctx, ch.Query{ - Body: query.String(), + Logger: zctx.From(ctx), + Body: query.String(), Result: proto.Results{ {Name: "name", Data: column}, }, @@ -323,6 +326,7 @@ func (p *promQuerier) queryPoints(ctx context.Context, query string) ([]storage. c = newPointColumns() ) if err := p.ch.Do(ctx, ch.Query{ + Logger: zctx.From(ctx), Body: query, Result: c.Result(), OnResult: func(ctx context.Context, block proto.Block) error { @@ -392,6 +396,7 @@ func (p *promQuerier) queryExpHistograms(ctx context.Context, query string) ([]s c = newExpHistogramColumns() ) if err := p.ch.Do(ctx, ch.Query{ + Logger: zctx.From(ctx), Body: query, Result: c.Result(), OnResult: func(ctx context.Context, block proto.Block) error { diff --git a/internal/chstorage/querier_traces.go b/internal/chstorage/querier_traces.go index 7729d5db..787b2917 100644 --- a/internal/chstorage/querier_traces.go +++ b/internal/chstorage/querier_traces.go @@ -104,7 +104,8 @@ func (q *Querier) TagNames(ctx context.Context) (r []string, rerr error) { data := new(proto.ColStr).LowCardinality() if err := q.ch.Do(ctx, ch.Query{ - Body: fmt.Sprintf("SELECT DISTINCT name FROM %#q", table), + Logger: zctx.From(ctx), + Body: fmt.Sprintf("SELECT DISTINCT name FROM %#q", table), Result: proto.ResultColumn{ Name: "name", Data: data, @@ -144,7 +145,8 @@ func (q *Querier) TagValues(ctx context.Context, tagName string) (_ iterators.It ) if err := q.ch.Do(ctx, ch.Query{ - Body: fmt.Sprintf("SELECT DISTINCT value, value_type FROM %#q WHERE name = %s", table, singleQuoted(tagName)), + Logger: zctx.From(ctx), + Body: fmt.Sprintf("SELECT DISTINCT value, value_type FROM %#q WHERE name = %s", table, singleQuoted(tagName)), Result: proto.Results{ {Name: "value", Data: &value}, {Name: "value_type", Data: proto.Wrap(&valueType, valueTypeDDL)}, @@ -432,6 +434,7 @@ func (q *Querier) querySpans(ctx context.Context, query string) (iterators.Itera var r []tracestorage.Span if err := q.ch.Do(ctx, ch.Query{ + Logger: zctx.From(ctx), Body: query, Result: c.Result(), OnResult: func(ctx context.Context, block proto.Block) (err error) { diff --git a/internal/chstorage/schema.go b/internal/chstorage/schema.go index cce394c1..0f81714e 100644 --- a/internal/chstorage/schema.go +++ b/internal/chstorage/schema.go @@ -6,6 +6,7 @@ import ( "github.com/ClickHouse/ch-go" "github.com/go-faster/errors" + "github.com/go-faster/sdk/zctx" ) // Tables define table names. @@ -103,7 +104,8 @@ func (t Tables) Create(ctx context.Context, c chClient) error { {t.LogAttrs, logAttrsSchema}, } { if err := c.Do(ctx, ch.Query{ - Body: fmt.Sprintf(s.query, s.name), + Logger: zctx.From(ctx), + Body: fmt.Sprintf(s.query, s.name), }); err != nil { return errors.Wrapf(err, "create %q", s.name) } From 4eb1179f708ee5f6f9860e5a776796db0ab8a46e Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 20:57:25 +0300 Subject: [PATCH 105/112] chore: pass ch.Query.Logger --- cmd/chotel/main.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cmd/chotel/main.go b/cmd/chotel/main.go index 12205d68..7e55e3c3 100644 --- a/cmd/chotel/main.go +++ b/cmd/chotel/main.go @@ -231,7 +231,8 @@ func (a *App) send(ctx context.Context, now time.Time) error { return errors.Wrap(err, "export") } if err := db.Do(ctx, ch.Query{ - Body: "INSERT INTO opentelemetry_span_export (trace_id, span_id, exported_at) VALUES", + Logger: zctx.From(ctx), + Body: "INSERT INTO opentelemetry_span_export (trace_id, span_id, exported_at) VALUES", Input: proto.Input{ {Name: "trace_id", Data: exported.TraceID}, {Name: "span_id", Data: exported.SpanID}, From a5eb3ebc1565e0edb36c3ea5e2d1887cb3e0d69b Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 20:57:59 +0300 Subject: [PATCH 106/112] chore: rm printf --- internal/chstorage/querier_logs.go | 8 -------- 1 file changed, 8 deletions(-) diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index e21587ac..d6d44231 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -403,8 +403,6 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta query.WriteByte(')') } - hasTraceID := false - for _, m := range params.Line { switch m.Op { case logql.OpEq, logql.OpRe: @@ -427,7 +425,6 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta switch len(v) { case len(otelstorage.TraceID{}): fmt.Fprintf(&query, " OR trace_id = unhex(%s)", singleQuoted(encoded)) - hasTraceID = true case len(otelstorage.SpanID{}): fmt.Fprintf(&query, " OR span_id = unhex(%s)", singleQuoted(encoded)) } @@ -438,10 +435,6 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta query.WriteByte(')') } - // TODO: use streaming. - if hasTraceID { - fmt.Println(query.String()) - } var data []logstorage.Record if err := q.ch.Do(ctx, ch.Query{ Logger: zctx.From(ctx).Named("ch"), @@ -458,6 +451,5 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta }); err != nil { return nil, errors.Wrap(err, "select") } - fmt.Println("got data", len(data)) return &logStaticIterator{data: data}, nil } From 77b135d648dd0b619b22824df6bd87415671a620 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 20:59:50 +0300 Subject: [PATCH 107/112] fix(chstorage): order by timestamp --- internal/chstorage/querier_logs.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index d6d44231..15f6350b 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -435,6 +435,8 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta query.WriteByte(')') } + query.WriteString(" ORDER BY timestamp") + var data []logstorage.Record if err := q.ch.Do(ctx, ch.Query{ Logger: zctx.From(ctx).Named("ch"), From 3286a7d56d52605aee0c0465c6f0d280e4fef5f4 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 21:04:48 +0300 Subject: [PATCH 108/112] chore: use lg from zctx --- cmd/oteldb/app.go | 4 +++- cmd/oteldb/oteldb.go | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/cmd/oteldb/app.go b/cmd/oteldb/app.go index d9d2a2ad..75fef413 100644 --- a/cmd/oteldb/app.go +++ b/cmd/oteldb/app.go @@ -8,6 +8,7 @@ import ( "time" "github.com/go-faster/errors" + "github.com/go-faster/sdk/zctx" "github.com/prometheus/prometheus/promql" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -37,10 +38,11 @@ type App struct { metrics Metrics } -func newApp(ctx context.Context, lg *zap.Logger, metrics Metrics) (_ *App, err error) { +func newApp(ctx context.Context, metrics Metrics) (_ *App, err error) { var ( storageType = strings.ToLower(os.Getenv("OTELDB_STORAGE")) m = NewMetricsOverride(metrics) + lg = zctx.From(ctx) app = &App{ services: map[string]func(context.Context) error{}, lg: lg, diff --git a/cmd/oteldb/oteldb.go b/cmd/oteldb/oteldb.go index 497c6ca7..95cb3400 100644 --- a/cmd/oteldb/oteldb.go +++ b/cmd/oteldb/oteldb.go @@ -77,7 +77,7 @@ func main() { ctx = zctx.Base(ctx, lg) } - root, err := newApp(ctx, lg, m) + root, err := newApp(ctx, m) if err != nil { return errors.Wrap(err, "setup") } From 9f36768e898c66760f95cbd1545de28e3b0c47d8 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Thu, 30 Nov 2023 21:40:34 +0300 Subject: [PATCH 109/112] feat(chstorage.querier.logs): optimize external table select --- internal/chstorage/querier_logs.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index 15f6350b..f2ec0215 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -164,11 +164,11 @@ func (q *Querier) getLabelMapping(ctx context.Context, labels []string) (_ map[s }) return nil }, - ExternalTable: "_labels", + ExternalTable: "labels", ExternalData: []proto.InputColumn{ {Name: "name", Data: &inputData}, }, - Body: fmt.Sprintf(`SELECT name, key FROM %[1]s INNER JOIN _labels ON (_labels.name = %[1]s.name)`, q.tables.LogAttrs), + Body: fmt.Sprintf(`SELECT name, key FROM %[1]s WHERE name IN labels`, q.tables.LogAttrs), }); err != nil { return nil, errors.Wrap(err, "select") } From 53127012fc805902a374b6b7754d4d6abd71c664 Mon Sep 17 00:00:00 2001 From: tdakkota Date: Fri, 1 Dec 2023 07:06:58 +0300 Subject: [PATCH 110/112] fix(chstorage.logs): use exact match for materialized `service.` labels --- internal/chstorage/querier_logs.go | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index f2ec0215..ae5b2509 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -24,8 +24,10 @@ import ( "github.com/go-faster/oteldb/internal/otelstorage" ) -var _ logstorage.Querier = (*Querier)(nil) -var _ logqlengine.Querier = (*Querier)(nil) +var ( + _ logstorage.Querier = (*Querier)(nil) + _ logqlengine.Querier = (*Querier)(nil) +) // LabelNames implements logstorage.Querier. func (q *Querier) LabelNames(ctx context.Context, opts logstorage.LabelsOptions) (_ []string, rerr error) { @@ -62,9 +64,9 @@ func (q *Querier) LabelNames(ctx context.Context, opts logstorage.LabelsOptions) } return nil }, - Body: fmt.Sprintf(`SELECT DISTINCT + Body: fmt.Sprintf(`SELECT DISTINCT arrayJoin(arrayConcat(JSONExtractKeys(attributes), JSONExtractKeys(resource), JSONExtractKeys(scope_attributes))) as key -FROM %s +FROM %s WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d) LIMIT 1000`, table, opts.Start, opts.End, @@ -241,13 +243,13 @@ func (q *Querier) LabelValues(ctx context.Context, labelName string, opts logsto } return nil }, - Body: fmt.Sprintf(`SELECT DISTINCT + Body: fmt.Sprintf(`SELECT DISTINCT array( - JSONExtractRaw(attributes, %[1]s), + JSONExtractRaw(attributes, %[1]s), JSONExtractRaw(scope_attributes, %[1]s), JSONExtractRaw(resource, %[1]s) ) as values -FROM %s +FROM %s WHERE (toUnixTimestamp64Nano(timestamp) >= %d AND toUnixTimestamp64Nano(timestamp) <= %d) LIMIT 1000`, singleQuoted(labelName), table, opts.Start, opts.End, ), @@ -377,7 +379,7 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta // Materialized from resource.service.{name,namespace,instance_id}. switch m.Op { case logql.OpEq, logql.OpNotEq: - fmt.Fprintf(&query, "positionUTF8(%s, %s) > 0", labelName, singleQuoted(m.Value)) + fmt.Fprintf(&query, "%s = %s", labelName, singleQuoted(m.Value)) case logql.OpRe, logql.OpNotRe: fmt.Fprintf(&query, "%s REGEXP %s", labelName, singleQuoted(m.Value)) } From e1f81a29aa6d23edc78d33dca246a1e29a85a530 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Fri, 1 Dec 2023 12:19:04 +0300 Subject: [PATCH 111/112] fix(chstorage.q.logs): fix body tag matcher --- integration/lokie2e/common_test.go | 5 +++++ internal/chstorage/querier_logs.go | 12 ++++++------ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/integration/lokie2e/common_test.go b/integration/lokie2e/common_test.go index 1046625f..36e4abca 100644 --- a/integration/lokie2e/common_test.go +++ b/integration/lokie2e/common_test.go @@ -147,12 +147,15 @@ func runTest( // By trace id. {`{trace_id="af36000000000000c517000000000003"}`, 1}, {`{trace_id="AF36000000000000C517000000000003"}`, 1}, + {`{trace_id=~"AF3600.+000C517000.+00003"}`, 1}, {`{trace_id="badbadbadbadbadbaddeadbeafbadbad"}`, 0}, + {`{trace_id=~"bad.+"}`, 0}, // By severity. {`{level="Info"}`, 121}, {`{level="INFO"}`, 121}, // All by service name. {`{service_name="testService"}`, len(set.Records)}, + {`{service_name=~"test.+"}`, len(set.Records)}, // Effectively match GET. {`{http_method="GET"}`, 21}, {`{http_method=~".*GET.*"}`, 21}, @@ -189,6 +192,8 @@ func runTest( {`{http_method=~".+"} |= "HEAD" |= " 500 "`, 2}, {`{http_method=~".+"} |~ "DELETE"`, 20}, {`{http_method=~".+"} |~ "HEAD" |= " 500 "`, 2}, + {`{http_method=~".+"} |~ "(GET|HEAD)"`, 43}, + {`{http_method=~".+"} |~ "GE.+"`, 21}, // Try to not use offloading. {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "DELETE"`, 20}, {`{http_method=~".+"} | line_format "{{ __line__ }}" |= "HEAD" |= " 500 "`, 2}, diff --git a/internal/chstorage/querier_logs.go b/internal/chstorage/querier_logs.go index ae5b2509..0fdfdb81 100644 --- a/internal/chstorage/querier_logs.go +++ b/internal/chstorage/querier_logs.go @@ -343,14 +343,14 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta case logql.OpEq, logql.OpNotEq: fmt.Fprintf(&query, "trace_id = unhex(%s)", singleQuoted(m.Value)) case logql.OpRe, logql.OpNotRe: - fmt.Fprintf(&query, "hex(trace_id) REGEXP %s", singleQuoted(m.Value)) + fmt.Fprintf(&query, "match(hex(trace_id), %s)", singleQuoted(m.Value)) } case logstorage.LabelSpanID: switch m.Op { case logql.OpEq, logql.OpNotEq: fmt.Fprintf(&query, "span_id = unhex(%s)", singleQuoted(m.Value)) case logql.OpRe, logql.OpNotRe: - fmt.Fprintf(&query, "hex(span_id) REGEXP %s", singleQuoted(m.Value)) + fmt.Fprintf(&query, "match(hex(span_id), %s)", singleQuoted(m.Value)) } case logstorage.LabelSeverity: switch m.Op { @@ -373,7 +373,7 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta case logql.OpEq, logql.OpNotEq: fmt.Fprintf(&query, "positionUTF8(body, %s) > 0", singleQuoted(m.Value)) case logql.OpRe, logql.OpNotRe: - fmt.Fprintf(&query, "service_name REGEXP %s", singleQuoted(m.Value)) + fmt.Fprintf(&query, "match(body, %s)", singleQuoted(m.Value)) } case logstorage.LabelServiceName, logstorage.LabelServiceNamespace, logstorage.LabelServiceInstanceID: // Materialized from resource.service.{name,namespace,instance_id}. @@ -381,7 +381,7 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta case logql.OpEq, logql.OpNotEq: fmt.Fprintf(&query, "%s = %s", labelName, singleQuoted(m.Value)) case logql.OpRe, logql.OpNotRe: - fmt.Fprintf(&query, "%s REGEXP %s", labelName, singleQuoted(m.Value)) + fmt.Fprintf(&query, "match(%s, %s)", labelName, singleQuoted(m.Value)) } default: // Search in all attributes. @@ -398,7 +398,7 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta case logql.OpEq, logql.OpNotEq: fmt.Fprintf(&query, "JSONExtractString(%s, %s) = %s", column, singleQuoted(labelName), singleQuoted(m.Value)) case logql.OpRe, logql.OpNotRe: - fmt.Fprintf(&query, "JSONExtractString(%s, %s) REGEXP %s", column, singleQuoted(labelName), singleQuoted(m.Value)) + fmt.Fprintf(&query, "match(JSONExtractString(%s, %s), %s)", column, singleQuoted(labelName), singleQuoted(m.Value)) } } } @@ -432,7 +432,7 @@ func (q *Querier) SelectLogs(ctx context.Context, start, end otelstorage.Timesta } } case logql.OpRe, logql.OpNotRe: - fmt.Fprintf(&query, "body REGEXP %s", singleQuoted(m.Value)) + fmt.Fprintf(&query, "match(body, %s)", singleQuoted(m.Value)) } query.WriteByte(')') } From 994861617c67d9da78f5c7e2e5d538ebc1b87de9 Mon Sep 17 00:00:00 2001 From: Aleksandr Razumov Date: Fri, 1 Dec 2023 12:53:09 +0300 Subject: [PATCH 112/112] fix(logqlengine): adapt to OTEL --- internal/logql/logqlengine/engine_test.go | 103 +++++++++++++++++---- internal/logql/logqlengine/label_set.go | 4 +- internal/logql/logqlengine/otel_adapter.go | 9 +- 3 files changed, 94 insertions(+), 22 deletions(-) diff --git a/internal/logql/logqlengine/engine_test.go b/internal/logql/logqlengine/engine_test.go index b0aa2ebf..d0ff0f3f 100644 --- a/internal/logql/logqlengine/engine_test.go +++ b/internal/logql/logqlengine/engine_test.go @@ -9,6 +9,7 @@ import ( "testing" "time" + "github.com/go-faster/jx" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/pdata/pcommon" @@ -56,13 +57,88 @@ func (m *mockQuerier) SelectLogs(_ context.Context, start, _ otelstorage.Timesta for _, l := range m.lines { ts = ts.Add(step) - records = append(records, logstorage.Record{ + body := l.line + rec := logstorage.Record{ Timestamp: otelstorage.NewTimestampFromTime(ts), Body: l.line, Attrs: otelstorage.Attrs(l.attrs), ScopeAttrs: otelstorage.Attrs(scopeAttrs), ResourceAttrs: otelstorage.Attrs(resAttrs), - }) + } + if rec.Attrs == otelstorage.Attrs(pcommon.Map{}) { + rec.Attrs = otelstorage.Attrs(pcommon.NewMap()) + } + if dec := jx.DecodeStr(body); dec.Next() == jx.Object { + rec.Body = "" + if err := dec.Obj(func(d *jx.Decoder, key string) error { + switch key { + case logstorage.LabelBody: + v, err := d.Str() + if err != nil { + return err + } + rec.Body = v + return nil + case logstorage.LabelTraceID: + v, err := d.Str() + if err != nil { + return err + } + traceID, err := otelstorage.ParseTraceID(v) + if err != nil { + return err + } + rec.TraceID = traceID + return nil + default: + switch d.Next() { + case jx.String: + v, err := d.Str() + if err != nil { + return err + } + rec.Attrs.AsMap().PutStr(key, v) + return nil + case jx.Bool: + v, err := d.Bool() + if err != nil { + return err + } + rec.Attrs.AsMap().PutBool(key, v) + return nil + case jx.Number: + v, err := d.Num() + if err != nil { + return err + } + if v.IsInt() { + n, err := v.Int64() + if err != nil { + return err + } + rec.Attrs.AsMap().PutInt(key, n) + } else { + n, err := v.Float64() + if err != nil { + return err + } + rec.Attrs.AsMap().PutDouble(key, n) + } + return nil + default: + v, err := d.Raw() + if err != nil { + return err + } + rec.Attrs.AsMap().PutStr(key, string(v)) + return nil + } + } + }); err != nil { + return nil, err + } + } + records = append(records, rec) } return iterators.Slice(records), nil @@ -128,20 +204,7 @@ func TestEngineEvalStream(t *testing.T) { { `{resource="test"}`, inputLines, - []resultLine{ - { - `{"id": 1, "foo": "4m", "bar": "1s", "baz": "1kb"}`, - map[string]string{}, - }, - { - `{"id": 2, "foo": "5m", "bar": "2s", "baz": "1mb"}`, - map[string]string{}, - }, - { - `{"id": 3, "foo": "6m", "bar": "3s", "baz": "1gb"}`, - map[string]string{}, - }, - }, + resultLines, false, }, { @@ -428,7 +491,13 @@ func TestEngineEvalStream(t *testing.T) { line: e.line, labels: e.labels, } - assert.Equal(t, result, tt.wantData[i]) + wanna := tt.wantData[i] + if jx.Valid([]byte(wanna.line)) { + assert.JSONEq(t, wanna.line, result.line) + } else { + assert.Equal(t, wanna.line, result.line) + } + assert.Equal(t, wanna.labels, result.labels) } }) } diff --git a/internal/logql/logqlengine/label_set.go b/internal/logql/logqlengine/label_set.go index aa1572a2..120fb6f9 100644 --- a/internal/logql/logqlengine/label_set.go +++ b/internal/logql/logqlengine/label_set.go @@ -88,7 +88,9 @@ func (l *LabelSet) SetFromRecord(record logstorage.Record) { if severity := record.SeverityNumber; severity != plog.SeverityNumberUnspecified { l.Set(logstorage.LabelSeverity, pcommon.NewValueStr(severity.String())) } - l.Set(logstorage.LabelBody, pcommon.NewValueStr(record.Body)) + if body := record.Body; body != "" { + l.Set(logstorage.LabelBody, pcommon.NewValueStr(body)) + } l.SetAttrs(record.Attrs, record.ScopeAttrs, record.ResourceAttrs) } diff --git a/internal/logql/logqlengine/otel_adapter.go b/internal/logql/logqlengine/otel_adapter.go index e5434d51..601e3ae1 100644 --- a/internal/logql/logqlengine/otel_adapter.go +++ b/internal/logql/logqlengine/otel_adapter.go @@ -12,10 +12,11 @@ func LineFromRecord(record logstorage.Record) string { // Create JSON object from record. e := &jx.Encoder{} e.Obj(func(e *jx.Encoder) { - e.Field(logstorage.LabelBody, func(e *jx.Encoder) { - e.Str(record.Body) - }) - + if len(record.Body) != 0 { + e.Field(logstorage.LabelBody, func(e *jx.Encoder) { + e.Str(record.Body) + }) + } if m := record.Attrs.AsMap(); m != (pcommon.Map{}) { record.Attrs.AsMap().Range(func(k string, v pcommon.Value) bool { e.Field(k, func(e *jx.Encoder) {