diff --git a/docs/sources/tempo/traceql/metrics-queries/_index.md b/docs/sources/tempo/traceql/metrics-queries/_index.md index 9f4c2a2fe77..817fd5dd005 100644 --- a/docs/sources/tempo/traceql/metrics-queries/_index.md +++ b/docs/sources/tempo/traceql/metrics-queries/_index.md @@ -60,7 +60,7 @@ Refer to [Solve problems using metrics queries](./solve-problems-metrics-queries ### Functions -TraceQL metrics queries currently include the following functions for aggregating over groups of spans: `rate`, `count_over_time`, `quantile_over_time`, `histogram_over_time`, and `compare`. +TraceQL metrics queries currently include the following functions for aggregating over groups of spans: `rate`, `count_over_time`, `max_over_time`, `min_over_time`, `avg_over_time`, `quantile_over_time`, `histogram_over_time`, and `compare`. These functions can be added as an operator at the end of any TraceQL query. For detailed information and example queries for each function, refer to [TraceQL metrics functions](./functions). diff --git a/docs/sources/tempo/traceql/metrics-queries/functions.md b/docs/sources/tempo/traceql/metrics-queries/functions.md index 2c20a34674f..392d2c92f20 100644 --- a/docs/sources/tempo/traceql/metrics-queries/functions.md +++ b/docs/sources/tempo/traceql/metrics-queries/functions.md @@ -12,7 +12,7 @@ keywords: -TraceQL supports `rate`, `count_over_time`, `quantile_over_time`, `histogram_over_time`, and `compare` functions. +TraceQL supports `rate`, `count_over_time`, `min_over_time`, `avg_over_time`, `quantile_over_time`, `histogram_over_time`, and `compare` functions. ## Available functions @@ -30,6 +30,9 @@ These functions can be added as an operator at the end of any TraceQL query. `max_over_time` : Returns the minimum value for the specified attribute across all matching spans per time interval (refer to the [`step` API parameter](https://grafana.com/docs/tempo//api_docs/#traceql-metrics)). +`avg_over_time` +: Returns the average value for the specified attribute across all matching spans per time interval (refer to the [`step` API parameter](https://grafana.com/docs/tempo//api_docs/#traceql-metrics)). + `quantile_over_time` : The quantile of the values in the specified interval @@ -94,7 +97,7 @@ This example counts the number of spans with name `"GET /:endpoint"` broken down ``` -## The `min_over_time` and `max_over_time` functions +## The `min_over_time`, `max_over_time`, and `avg_over_time` functions The `min_over_time()` function lets you aggregate numerical attributes by calculating their minimum value. For example, you could choose to calculate the minimum duration of a group of spans, or you could choose to calculate the minimum value of a custom attribute you've attached to your spans, like `span.shopping.cart.entries`. @@ -103,11 +106,14 @@ The time interval that the minimum is computed over is set by the `step` paramet The `max_over_time()` let you aggregate numerical values by computing the maximum value of them, such as the all important span duration. The time interval that the maximum is computer over is set by the `step` parameter. +The `avg_over_time()` function lets you aggregate numerical values by computing the maximum value of them, such as the all important span duration. +The time interval that the maximum is computer over is set by the `step` parameter. + For more information, refer to the [`step` API parameter](https://grafana.com/docs/tempo//api_docs/#traceql-metrics). ### Parameters -Numerical field that you want to calculate the minimum or maximum of. +Numerical field that you want to calculate the minimum, maximum, or average of. ### Examples @@ -134,6 +140,16 @@ This example computes the maximum duration for each `http.target` of all spans n { name = "GET /:endpoint" } | max_over_time(span.http.response.size) ``` +This example computes the average duration for each `http.status_code` of all spans named `"GET /:endpoint"`. + +``` +{ name = "GET /:endpoint" } | avg_over_time(duration) by (span.http.status_code) +``` + +``` +{ name = "GET /:endpoint" } | avg_over_time(span.http.response.size) +``` + ## The `quantile_over_time` and `histogram_over_time` functions The `quantile_over_time()` and `histogram_over_time()` functions let you aggregate numerical values, such as the all important span duration. diff --git a/pkg/traceql/engine_metrics.go b/pkg/traceql/engine_metrics.go index b94eeabc455..3aba0a74f6c 100644 --- a/pkg/traceql/engine_metrics.go +++ b/pkg/traceql/engine_metrics.go @@ -17,6 +17,8 @@ import ( ) const ( + internalLabelMetaType = "__meta_type" + internalMetaTypeCount = "__count" internalLabelBucket = "__bucket" maxExemplars = 100 maxExemplarsPerBucket = 2 @@ -176,10 +178,12 @@ func (ls Labels) String() string { promValue = "" case l.Value.Type == TypeString: s := l.Value.EncodeToString(false) - if s != "" { - promValue = s - } else { + if s == "nil" { + promValue = "" + } else if s == "" { promValue = "" + } else { + promValue = s } default: promValue = l.Value.EncodeToString(false) diff --git a/pkg/traceql/engine_metrics_average.go b/pkg/traceql/engine_metrics_average.go new file mode 100644 index 00000000000..0b6d934a7cd --- /dev/null +++ b/pkg/traceql/engine_metrics_average.go @@ -0,0 +1,554 @@ +package traceql + +import ( + "fmt" + "math" + "strings" + "time" + + "github.com/grafana/tempo/pkg/tempopb" + v1 "github.com/grafana/tempo/pkg/tempopb/common/v1" + "github.com/prometheus/prometheus/model/labels" +) + +// Average over time aggregator +type averageOverTimeAggregator struct { + by []Attribute + attr Attribute + // Average over time span aggregator + agg SpanAggregator + // Average over time series aggregator + seriesAgg SeriesAggregator + exemplarFn getExemplar + mode AggregateMode +} + +var _ metricsFirstStageElement = (*averageOverTimeAggregator)(nil) + +func newAverageOverTimeMetricsAggregator(attr Attribute, by []Attribute) *averageOverTimeAggregator { + return &averageOverTimeAggregator{ + attr: attr, + by: by, + } +} + +func (a *averageOverTimeAggregator) init(q *tempopb.QueryRangeRequest, mode AggregateMode) { + exemplarFn := func(s Span) (float64, uint64) { + return math.NaN(), a.spanStartTimeMs(s) + } + + a.seriesAgg = &averageOverTimeSeriesAggregator{ + weightedAverageSeries: make(map[string]averageSeries), + len: IntervalCount(q.Start, q.End, q.Step), + start: q.Start, + end: q.End, + step: q.Step, + exemplarBuckets: newBucketSet(IntervalCount(q.Start, q.End, q.Step)), + } + + if mode == AggregateModeRaw { + a.agg = newAvgOverTimeSpanAggregator(a.attr, a.by, q.Start, q.End, q.Step) + } + + a.exemplarFn = exemplarFn + a.mode = mode +} + +func (a *averageOverTimeAggregator) observe(span Span) { + a.agg.Observe(span) +} + +func (a *averageOverTimeAggregator) observeExemplar(span Span) { + v, ts := a.exemplarFn(span) + a.agg.ObserveExemplar(span, v, ts) +} + +func (a *averageOverTimeAggregator) observeSeries(ss []*tempopb.TimeSeries) { + a.seriesAgg.Combine(ss) +} + +func (a *averageOverTimeAggregator) result() SeriesSet { + if a.agg != nil { + return a.agg.Series() + } + + // In the frontend-version the results come from + // the job-level aggregator + ss := a.seriesAgg.Results() + if a.mode == AggregateModeFinal { + for i := range ss { + if strings.Contains(i, internalLabelMetaType) { + delete(ss, i) + } + } + } + return ss +} + +func (a *averageOverTimeAggregator) extractConditions(request *FetchSpansRequest) { + // For metrics aggregators based on a span attribute we have to include it + includeAttribute := a.attr != (Attribute{}) && !request.HasAttribute(a.attr) + if includeAttribute { + request.SecondPassConditions = append(request.SecondPassConditions, Condition{ + Attribute: a.attr, + }) + } + + for _, b := range a.by { + if !request.HasAttribute(b) { + request.SecondPassConditions = append(request.SecondPassConditions, Condition{ + Attribute: b, + }) + } + } +} + +func (a *averageOverTimeAggregator) validate() error { + if len(a.by) >= maxGroupBys { + return newUnsupportedError(fmt.Sprintf("metrics group by %v values", len(a.by))) + } + return nil +} + +func (a *averageOverTimeAggregator) spanStartTimeMs(s Span) uint64 { + return s.StartTimeUnixNanos() / uint64(time.Millisecond) +} + +func (a *averageOverTimeAggregator) String() string { + s := strings.Builder{} + + s.WriteString(metricsAggregateAvgOverTime.String()) + s.WriteString("(") + if a.attr != (Attribute{}) { + s.WriteString(a.attr.String()) + } + s.WriteString(")") + + if len(a.by) > 0 { + s.WriteString("by(") + for i, b := range a.by { + s.WriteString(b.String()) + if i < len(a.by)-1 { + s.WriteString(",") + } + } + s.WriteString(")") + } + return s.String() +} + +type averageOverTimeSeriesAggregator struct { + weightedAverageSeries map[string]averageSeries + len int + start, end, step uint64 + exemplarBuckets *bucketSet +} + +type averageValue struct { + mean float64 + compensation float64 + weight float64 +} + +// Adds an increment to the existing mean using Kahan sumnmation algorithm. +// The compensation is accumulated and not applied to reduce the error +func (a *averageValue) add(inc float64) { + if math.IsInf(a.mean, 0) { + if math.IsInf(inc, 0) && (a.mean > 0) == (inc > 0) { + // The `mean` and `ic` values are `Inf` of the same sign. They + // can't be subtracted, but the value of `mean` is correct + // already. + return + } + if !math.IsInf(inc, 0) && !math.IsNaN(inc) { + // At this stage, the mean is an infinite. If the added + // value is neither an Inf or a Nan, we can keep that mean + // value. + return + } + } + val, c := kahanSumInc(inc, a.mean, a.compensation) + a.mean = val + a.compensation = c +} + +func kahanSumInc(inc, sum, c float64) (newSum, newC float64) { + t := sum + inc + switch { + case math.IsInf(t, 0): + c = 0 + + // Using Neumaier improvement, swap if next term larger than sum. + case math.Abs(sum) >= math.Abs(inc): + c += (sum - t) + inc + default: + c += (inc - t) + sum + } + return t, c +} + +type averageSeries struct { + values []averageValue + labels Labels + Exemplars []Exemplar +} + +func newAverageSeries(len int, lenExemplars int, labels Labels) averageSeries { + s := averageSeries{ + values: make([]averageValue, len), + labels: labels, + Exemplars: make([]Exemplar, 0, lenExemplars), + } + // Init to nan to discriminate uninitialized values from 0 + for i := range s.values { + s.values[i].mean = nan + s.values[i].weight = nan + } + return s +} + +// it adds the compensation to the final value to retain precission +func (k *averageSeries) getAvgSeries() TimeSeries { + ts := TimeSeries{ + Labels: k.labels, + Values: make([]float64, len(k.values)), + Exemplars: k.Exemplars, + } + + for i, v := range k.values { + ts.Values[i] = v.mean + v.compensation + } + return ts +} + +func (k *averageSeries) getCountSeries() TimeSeries { + countLabels := append(k.labels, Label{internalLabelMetaType, NewStaticString(internalMetaTypeCount)}) + ts := TimeSeries{ + Labels: countLabels, + Values: make([]float64, len(k.values)), + } + for i, v := range k.values { + ts.Values[i] = v.weight + } + return ts +} + +// It increments the mean based on a new value +func (k *averageSeries) addIncrementMean(interval int, inc float64) { + currentMean := k.values[interval] + if math.IsNaN(currentMean.mean) && !math.IsNaN(inc) { + k.values[interval] = averageValue{mean: inc, weight: 1} + return + } + currentMean.weight++ + currentMean.add(inc/currentMean.weight - currentMean.mean/currentMean.weight) + k.values[interval] = currentMean +} + +// It calculates the incremental weighted mean using kahan-neumaier summation and a delta approach. +// By adding incremental values we prevent overflow +func (k *averageSeries) addWeigthedMean(interval int, mean float64, weight float64) { + currentMean := k.values[interval] + if math.IsNaN(currentMean.mean) && !math.IsNaN(mean) { + k.values[interval] = averageValue{mean: mean, weight: weight} + return + } + + sumWeights := currentMean.weight + weight + meanDelta := ((mean - currentMean.mean) * weight) / sumWeights + meanDelta -= currentMean.compensation + + currentMean.add(meanDelta) + k.values[interval] = currentMean +} + +var ( + _ SeriesAggregator = (*averageOverTimeSeriesAggregator)(nil) + nan = math.Float64frombits(normalNaN) +) + +func (b *averageOverTimeSeriesAggregator) Combine(in []*tempopb.TimeSeries) { + // We traverse the TimeSeries to initialize new TimeSeries and map the counter series with the position in the `in` array + countPosMapper := make(map[string]int, len(in)/2) + for i, ts := range in { + _, ok := b.weightedAverageSeries[ts.PromLabels] + if strings.Contains(ts.PromLabels, internalLabelMetaType) { + // Label series without the count metatype, this will match with its average series + avgSeriesPromLabel := getLabels(ts.Labels, internalLabelMetaType).String() + // mapping of the position of the count series in the time series array + countPosMapper[avgSeriesPromLabel] = i + } else if !ok { + promLabels := getLabels(ts.Labels, "") + b.weightedAverageSeries[ts.PromLabels] = newAverageSeries(b.len, len(ts.Exemplars), promLabels) + } + } + for _, ts := range in { + existing, ok := b.weightedAverageSeries[ts.PromLabels] + if !ok { + // This is a counter series, we can skip it + continue + } + for i, sample := range ts.Samples { + pos := IntervalOfMs(sample.TimestampMs, b.start, b.end, b.step) + if pos < 0 || pos >= len(b.weightedAverageSeries[ts.PromLabels].values) { + continue + } + + incomingMean := sample.Value + incomingWeight := in[countPosMapper[ts.PromLabels]].Samples[i].Value + existing.addWeigthedMean(pos, incomingMean, incomingWeight) + b.aggregateExemplars(ts, b.weightedAverageSeries[ts.PromLabels]) + } + } +} + +func (b *averageOverTimeSeriesAggregator) aggregateExemplars(ts *tempopb.TimeSeries, existing averageSeries) { + for _, exemplar := range ts.Exemplars { + if b.exemplarBuckets.testTotal() { + break + } + interval := IntervalOfMs(exemplar.TimestampMs, b.start, b.end, b.step) + if b.exemplarBuckets.addAndTest(interval) { + continue // Skip this exemplar and continue, next exemplar might fit in a different bucket } + } + labels := make(Labels, 0, len(exemplar.Labels)) + for _, l := range exemplar.Labels { + labels = append(labels, Label{ + Name: l.Key, + Value: StaticFromAnyValue(l.Value), + }) + } + value := exemplar.Value + if math.IsNaN(value) { + value = 0 // TODO: Use the value of the series at the same timestamp + } + existing.Exemplars = append(existing.Exemplars, Exemplar{ + Labels: labels, + Value: value, + TimestampMs: uint64(exemplar.TimestampMs), + }) + } +} + +func getLabels(vals []v1.KeyValue, skipKey string) Labels { + labels := make(Labels, 0, len(vals)) + for _, l := range vals { + if skipKey != "" && l.Key == skipKey { + continue + } + labels = append(labels, Label{ + Name: l.Key, + Value: StaticFromAnyValue(l.Value), + }) + } + return labels +} + +func (b *averageOverTimeSeriesAggregator) Results() SeriesSet { + ss := SeriesSet{} + for k, v := range b.weightedAverageSeries { + ss[k] = v.getAvgSeries() + countSeries := v.getCountSeries() + ss[countSeries.Labels.String()] = countSeries + } + return ss +} + +// Accumulated results of average over time +type avgOverTimeSeries[S StaticVals] struct { + average averageSeries + exemplarBuckets *bucketSet + vals S + initialized bool +} + +// In charge of calculating the average over time for a set of spans +// First aggregation layer +type avgOverTimeSpanAggregator[F FastStatic, S StaticVals] struct { + // Config + by []Attribute // Original attributes: .foo + byLookups [][]Attribute // Lookups: span.foo resource.foo + getSpanAttValue func(s Span) float64 + start uint64 + end uint64 + step uint64 + + // Data + series map[F]avgOverTimeSeries[S] + lastSeries avgOverTimeSeries[S] + buf fastStaticWithValues[F, S] + lastBuf fastStaticWithValues[F, S] +} + +var _ SpanAggregator = (*avgOverTimeSpanAggregator[FastStatic1, StaticVals1])(nil) + +func newAvgOverTimeSpanAggregator(attr Attribute, by []Attribute, start, end, step uint64) SpanAggregator { + lookups := make([][]Attribute, len(by)) + for i, attr := range by { + if attr.Intrinsic == IntrinsicNone && attr.Scope == AttributeScopeNone { + // Unscoped attribute. Check span-level, then resource-level. + // TODO - Is this taken care of by span.AttributeFor now? + lookups[i] = []Attribute{ + NewScopedAttribute(AttributeScopeSpan, false, attr.Name), + NewScopedAttribute(AttributeScopeResource, false, attr.Name), + } + } else { + lookups[i] = []Attribute{attr} + } + } + + aggNum := len(lookups) + + switch aggNum { + case 2: + return newAvgAggregator[FastStatic2, StaticVals2](attr, by, lookups, start, end, step) + case 3: + return newAvgAggregator[FastStatic3, StaticVals3](attr, by, lookups, start, end, step) + case 4: + return newAvgAggregator[FastStatic4, StaticVals4](attr, by, lookups, start, end, step) + case 5: + return newAvgAggregator[FastStatic5, StaticVals5](attr, by, lookups, start, end, step) + default: + return newAvgAggregator[FastStatic1, StaticVals1](attr, by, lookups, start, end, step) + } +} + +func newAvgAggregator[F FastStatic, S StaticVals](attr Attribute, by []Attribute, lookups [][]Attribute, start, end, step uint64) SpanAggregator { + var fn func(s Span) float64 + + switch attr { + case IntrinsicDurationAttribute: + fn = func(s Span) float64 { + return float64(s.DurationNanos()) / float64(time.Second) + } + default: + fn = func(s Span) float64 { + f, a := FloatizeAttribute(s, attr) + if a == TypeNil { + return math.Float64frombits(normalNaN) + } + return f + } + } + + return &avgOverTimeSpanAggregator[F, S]{ + series: map[F]avgOverTimeSeries[S]{}, + getSpanAttValue: fn, + by: by, + byLookups: lookups, + start: start, + end: end, + step: step, + } +} + +func (g *avgOverTimeSpanAggregator[F, S]) Observe(span Span) { + interval := IntervalOf(span.StartTimeUnixNanos(), g.start, g.end, g.step) + if interval == -1 { + return + } + + inc := g.getSpanAttValue(span) + if math.IsNaN(inc) { + return + } + + s := g.getSeries(span) + s.average.addIncrementMean(interval, inc) +} + +func (g *avgOverTimeSpanAggregator[F, S]) ObserveExemplar(span Span, value float64, ts uint64) { + s := g.getSeries(span) + if s.exemplarBuckets.testTotal() { + return + } + interval := IntervalOfMs(int64(ts), g.start, g.end, g.step) + if s.exemplarBuckets.addAndTest(interval) { + return + } + + all := span.AllAttributes() + lbls := make(Labels, 0, len(all)) + for k, v := range span.AllAttributes() { + lbls = append(lbls, Label{k.String(), v}) + } + + s.average.Exemplars = append(s.average.Exemplars, Exemplar{ + Labels: lbls, + Value: value, + TimestampMs: ts, + }) + g.series[g.buf.fast] = s +} + +func (g *avgOverTimeSpanAggregator[F, S]) labelsFor(vals S) (Labels, string) { + if g.by == nil { + serieLabel := make(Labels, 1, 2) + serieLabel[0] = Label{labels.MetricName, NewStaticString(metricsAggregateAvgOverTime.String())} + return serieLabel, serieLabel.String() + } + labels := make(Labels, 0, len(g.by)+1) + for i := range g.by { + if vals[i].Type == TypeNil { + continue + } + labels = append(labels, Label{g.by[i].String(), vals[i]}) + } + + if len(labels) == 0 { + // When all nil then force one + labels = append(labels, Label{g.by[0].String(), NewStaticNil()}) + } + + return labels, labels.String() +} + +func (g *avgOverTimeSpanAggregator[F, S]) Series() SeriesSet { + ss := SeriesSet{} + + for _, s := range g.series { + labels, promLabelsAvg := g.labelsFor(s.vals) + s.average.labels = labels + // Average series + averageSeries := s.average.getAvgSeries() + // Count series + countSeries := s.average.getCountSeries() + + ss[promLabelsAvg] = averageSeries + ss[countSeries.Labels.String()] = countSeries + } + + return ss +} + +// getSeries gets the series for the current span. +// It will reuse the last series if possible. +func (g *avgOverTimeSpanAggregator[F, S]) getSeries(span Span) avgOverTimeSeries[S] { + // Get Grouping values + for i, lookups := range g.byLookups { + val := lookup(lookups, span) + g.buf.vals[i] = val + g.buf.fast[i] = val.MapKey() + } + + // Fast path + if g.lastBuf.fast == g.buf.fast && g.lastSeries.initialized { + return g.lastSeries + } + + s, ok := g.series[g.buf.fast] + if !ok { + intervals := IntervalCount(g.start, g.end, g.step) + s = avgOverTimeSeries[S]{ + vals: g.buf.vals, + average: newAverageSeries(intervals, maxExemplars, nil), + exemplarBuckets: newBucketSet(intervals), + initialized: true, + } + g.series[g.buf.fast] = s + } + + g.lastBuf = g.buf + g.lastSeries = s + return s +} diff --git a/pkg/traceql/engine_metrics_compare.go b/pkg/traceql/engine_metrics_compare.go index 292b3e13785..37ab60de569 100644 --- a/pkg/traceql/engine_metrics_compare.go +++ b/pkg/traceql/engine_metrics_compare.go @@ -10,7 +10,6 @@ import ( ) const ( - internalLabelMetaType = "__meta_type" internalMetaTypeBaseline = "baseline" internalMetaTypeSelection = "selection" internalMetaTypeBaselineTotal = "baseline_total" diff --git a/pkg/traceql/engine_metrics_test.go b/pkg/traceql/engine_metrics_test.go index 0656de28a9e..2fc0e356896 100644 --- a/pkg/traceql/engine_metrics_test.go +++ b/pkg/traceql/engine_metrics_test.go @@ -617,6 +617,253 @@ func TestMinOverTimeForSpanAttribute(t *testing.T) { } } +func TestAvgOverTimeForDuration(t *testing.T) { + req := &tempopb.QueryRangeRequest{ + Start: uint64(1 * time.Second), + End: uint64(3 * time.Second), + Step: uint64(1 * time.Second), + Query: "{ } | avg_over_time(duration) by (span.foo)", + } + + // A variety of spans across times, durations, and series. All durations are powers of 2 for simplicity + in := []Span{ + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithDuration(500), + + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithDuration(200), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithDuration(300), + } + + result := runTraceQLMetric(t, req, in) + + fooBaz := result[`{span.foo="baz"}`] + fooBar := result[`{span.foo="bar"}`] + + // We cannot compare with require.Equal because NaN != NaN + assert.True(t, math.IsNaN(fooBaz.Values[0])) + assert.True(t, math.IsNaN(fooBaz.Values[1])) + assert.Equal(t, 200., fooBaz.Values[2]*float64(time.Second)) + + assert.Equal(t, 100., fooBar.Values[0]*float64(time.Second)) + assert.Equal(t, 200., fooBar.Values[1]*float64(time.Second)) + assert.True(t, math.IsNaN(fooBar.Values[2])) +} + +func TestAvgOverTimeForDurationWithoutAggregation(t *testing.T) { + req := &tempopb.QueryRangeRequest{ + Start: uint64(1 * time.Second), + End: uint64(3 * time.Second), + Step: uint64(1 * time.Second), + Query: "{ } | avg_over_time(duration)", + } + + // A variety of spans across times, durations, and series. All durations are powers of 2 for simplicity + in := []Span{ + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithDuration(500), + + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithDuration(100), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithDuration(200), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "bar").WithDuration(300), + } + + result := runTraceQLMetric(t, req, in) + avg := result[`{__name__="avg_over_time"}`] + + assert.Equal(t, 100., avg.Values[0]*float64(time.Second)) + assert.Equal(t, 200., avg.Values[1]*float64(time.Second)) + assert.Equal(t, 200., avg.Values[2]*float64(time.Second)) +} + +func TestAvgOverTimeForSpanAttribute(t *testing.T) { + req := &tempopb.QueryRangeRequest{ + Start: uint64(1 * time.Second), + End: uint64(3 * time.Second), + Step: uint64(1 * time.Second), + Query: "{ } | avg_over_time(span.http.status_code) by (span.foo)", + } + + // A variety of spans across times, durations, and series. All durations are powers of 2 for simplicity + in := []Span{ + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(128), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 404).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(512), + + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(64), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(8), + + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 200).WithDuration(512), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 400).WithDuration(1024), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 300).WithDuration(512), + } + + in2 := []Span{ + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(128), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(512), + + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(64), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(8), + + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 200).WithDuration(512), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 200).WithDuration(1024), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 200).WithDuration(512), + } + + result := runTraceQLMetric(t, req, in, in2) + + fooBaz := result[`{span.foo="baz"}`] + fooBar := result[`{span.foo="bar"}`] + + // Alas,we cannot compare with require.Equal because NaN != NaN + // foo.baz = (NaN, NaN, 250) + assert.True(t, math.IsNaN(fooBaz.Values[0])) + assert.True(t, math.IsNaN(fooBaz.Values[1])) + assert.Equal(t, 250.0, fooBaz.Values[2]) + + // foo.bar = (234,200, NaN) + assert.Equal(t, 234.0, fooBar.Values[0]) + assert.Equal(t, 200.0, fooBar.Values[1]) + assert.True(t, math.IsNaN(fooBar.Values[2])) + + // Test that NaN values are not included in the samples after casting to proto + ts := result.ToProto(req) + fooBarSamples := []tempopb.Sample{{TimestampMs: 1000, Value: 234}, {TimestampMs: 2000, Value: 200}} + fooBazSamples := []tempopb.Sample{{TimestampMs: 3000, Value: 250}} + + for _, s := range ts { + if s.PromLabels == "{span.foo=\"bar\"}" { + assert.Equal(t, fooBarSamples, s.Samples) + } else { + assert.Equal(t, fooBazSamples, s.Samples) + } + } +} + +func TestAvgOverTimeWithNoMatch(t *testing.T) { + req := &tempopb.QueryRangeRequest{ + Start: uint64(1 * time.Second), + End: uint64(3 * time.Second), + Step: uint64(1 * time.Second), + Query: "{ } | avg_over_time(span.buu)", + } + + // A variety of spans across times, durations, and series. All durations are powers of 2 for simplicity + in := []Span{ + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(128), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 404).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(512), + + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(64), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(256), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200).WithDuration(8), + + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 201).WithDuration(512), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 401).WithDuration(1024), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 500).WithDuration(512), + } + + result := runTraceQLMetric(t, req, in) + + // Test that empty timeseries are not included + ts := result.ToProto(req) + + assert.True(t, len(ts) == 0) +} + +func TestObserveSeriesAverageOverTimeForSpanAttribute(t *testing.T) { + req := &tempopb.QueryRangeRequest{ + Start: uint64(1 * time.Second), + End: uint64(3 * time.Second), + Step: uint64(1 * time.Second), + Query: "{ } | avg_over_time(span.http.status_code) by (span.foo)", + } + + // A variety of spans across times, durations, and series. All durations are powers of 2 for simplicity + in := []Span{ + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 300), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 400), + + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 100), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 100), + + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 200), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 400), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 500), + } + + in2 := []Span{ + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 100), + newMockSpan(nil).WithStartTime(uint64(1*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 300), + + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 400), + newMockSpan(nil).WithStartTime(uint64(2*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 200), + + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "baz").WithSpanInt("http.status_code", 100), + newMockSpan(nil).WithStartTime(uint64(3*time.Second)).WithSpanString("foo", "bar").WithSpanInt("http.status_code", 100), + } + + e := NewEngine() + layer1A, _ := e.CompileMetricsQueryRange(req, 0, 0, false) + layer1B, _ := e.CompileMetricsQueryRange(req, 0, 0, false) + layer2A, _ := e.CompileMetricsQueryRangeNonRaw(req, AggregateModeSum) + layer2B, _ := e.CompileMetricsQueryRangeNonRaw(req, AggregateModeSum) + layer3, _ := e.CompileMetricsQueryRangeNonRaw(req, AggregateModeFinal) + + for _, s := range in { + layer1A.metricsPipeline.observe(s) + } + + layer2A.ObserveSeries(layer1A.Results().ToProto(req)) + + for _, s := range in2 { + layer1B.metricsPipeline.observe(s) + } + + layer2B.ObserveSeries(layer1B.Results().ToProto(req)) + + layer3.ObserveSeries(layer2A.Results().ToProto(req)) + layer3.ObserveSeries(layer2B.Results().ToProto(req)) + + result := layer3.Results() + + fooBaz := result[`{span.foo="baz"}`] + fooBar := result[`{span.foo="bar"}`] + + // Alas,we cannot compare with require.Equal because NaN != NaN + // foo.baz = (NaN, NaN, 300) + assert.True(t, math.IsNaN(fooBaz.Values[0])) + assert.True(t, math.IsNaN(fooBaz.Values[1])) + // 300 = (200 + 400 + 500 + 100) / 4 + assert.Equal(t, 300.0, fooBaz.Values[2]) + + // foo.bar = (260,200, 100) + assert.Equal(t, 260.0, fooBar.Values[0]) + assert.Equal(t, 200.0, fooBar.Values[1]) + assert.Equal(t, 100.0, fooBar.Values[2]) +} + func TestMaxOverTimeForDuration(t *testing.T) { req := &tempopb.QueryRangeRequest{ Start: uint64(1 * time.Second), @@ -856,7 +1103,6 @@ func runTraceQLMetric(t *testing.T, req *tempopb.QueryRangeRequest, inSpans ...[ // These are summed counts over time by bucket res := layer2.Results() layer3.ObserveSeries(res.ToProto(req)) - // Layer 3 final results return layer3.Results() diff --git a/pkg/traceql/enum_aggregates.go b/pkg/traceql/enum_aggregates.go index b5169bf486d..89afe173e8a 100644 --- a/pkg/traceql/enum_aggregates.go +++ b/pkg/traceql/enum_aggregates.go @@ -56,6 +56,7 @@ const ( metricsAggregateCountOverTime metricsAggregateMinOverTime metricsAggregateMaxOverTime + metricsAggregateAvgOverTime metricsAggregateQuantileOverTime metricsAggregateHistogramOverTime ) @@ -70,6 +71,8 @@ func (a MetricsAggregateOp) String() string { return "min_over_time" case metricsAggregateMaxOverTime: return "max_over_time" + case metricsAggregateAvgOverTime: + return "avg_over_time" case metricsAggregateQuantileOverTime: return "quantile_over_time" case metricsAggregateHistogramOverTime: diff --git a/pkg/traceql/expr.y b/pkg/traceql/expr.y index b0c1e6a35a6..538c6e1209a 100644 --- a/pkg/traceql/expr.y +++ b/pkg/traceql/expr.y @@ -100,7 +100,7 @@ import ( COUNT AVG MAX MIN SUM BY COALESCE SELECT END_ATTRIBUTE - RATE COUNT_OVER_TIME MIN_OVER_TIME MAX_OVER_TIME QUANTILE_OVER_TIME HISTOGRAM_OVER_TIME COMPARE + RATE COUNT_OVER_TIME MIN_OVER_TIME MAX_OVER_TIME AVG_OVER_TIME QUANTILE_OVER_TIME HISTOGRAM_OVER_TIME COMPARE WITH // Operators are listed with increasing precedence. @@ -302,6 +302,8 @@ metricsAggregation: | MIN_OVER_TIME OPEN_PARENS attribute CLOSE_PARENS BY OPEN_PARENS attributeList CLOSE_PARENS { $$ = newMetricsAggregateWithAttr(metricsAggregateMinOverTime, $3, $7) } | MAX_OVER_TIME OPEN_PARENS attribute CLOSE_PARENS { $$ = newMetricsAggregateWithAttr(metricsAggregateMaxOverTime, $3, nil) } | MAX_OVER_TIME OPEN_PARENS attribute CLOSE_PARENS BY OPEN_PARENS attributeList CLOSE_PARENS { $$ = newMetricsAggregateWithAttr(metricsAggregateMaxOverTime, $3, $7) } + | AVG_OVER_TIME OPEN_PARENS attribute CLOSE_PARENS { $$ = newAverageOverTimeMetricsAggregator($3, nil) } + | AVG_OVER_TIME OPEN_PARENS attribute CLOSE_PARENS BY OPEN_PARENS attributeList CLOSE_PARENS { $$ = newAverageOverTimeMetricsAggregator($3, $7) } | QUANTILE_OVER_TIME OPEN_PARENS attribute COMMA numericList CLOSE_PARENS { $$ = newMetricsAggregateQuantileOverTime($3, $5, nil) } | QUANTILE_OVER_TIME OPEN_PARENS attribute COMMA numericList CLOSE_PARENS BY OPEN_PARENS attributeList CLOSE_PARENS { $$ = newMetricsAggregateQuantileOverTime($3, $5, $9) } | HISTOGRAM_OVER_TIME OPEN_PARENS attribute CLOSE_PARENS { $$ = newMetricsAggregateWithAttr(metricsAggregateHistogramOverTime, $3, nil) } diff --git a/pkg/traceql/expr.y.go b/pkg/traceql/expr.y.go index 645193e7538..f8855be5682 100644 --- a/pkg/traceql/expr.y.go +++ b/pkg/traceql/expr.y.go @@ -121,40 +121,41 @@ const RATE = 57408 const COUNT_OVER_TIME = 57409 const MIN_OVER_TIME = 57410 const MAX_OVER_TIME = 57411 -const QUANTILE_OVER_TIME = 57412 -const HISTOGRAM_OVER_TIME = 57413 -const COMPARE = 57414 -const WITH = 57415 -const PIPE = 57416 -const AND = 57417 -const OR = 57418 -const EQ = 57419 -const NEQ = 57420 -const LT = 57421 -const LTE = 57422 -const GT = 57423 -const GTE = 57424 -const NRE = 57425 -const RE = 57426 -const DESC = 57427 -const ANCE = 57428 -const SIBL = 57429 -const NOT_CHILD = 57430 -const NOT_PARENT = 57431 -const NOT_DESC = 57432 -const NOT_ANCE = 57433 -const UNION_CHILD = 57434 -const UNION_PARENT = 57435 -const UNION_DESC = 57436 -const UNION_ANCE = 57437 -const UNION_SIBL = 57438 -const ADD = 57439 -const SUB = 57440 -const NOT = 57441 -const MUL = 57442 -const DIV = 57443 -const MOD = 57444 -const POW = 57445 +const AVG_OVER_TIME = 57412 +const QUANTILE_OVER_TIME = 57413 +const HISTOGRAM_OVER_TIME = 57414 +const COMPARE = 57415 +const WITH = 57416 +const PIPE = 57417 +const AND = 57418 +const OR = 57419 +const EQ = 57420 +const NEQ = 57421 +const LT = 57422 +const LTE = 57423 +const GT = 57424 +const GTE = 57425 +const NRE = 57426 +const RE = 57427 +const DESC = 57428 +const ANCE = 57429 +const SIBL = 57430 +const NOT_CHILD = 57431 +const NOT_PARENT = 57432 +const NOT_DESC = 57433 +const NOT_ANCE = 57434 +const UNION_CHILD = 57435 +const UNION_PARENT = 57436 +const UNION_DESC = 57437 +const UNION_ANCE = 57438 +const UNION_SIBL = 57439 +const ADD = 57440 +const SUB = 57441 +const NOT = 57442 +const MUL = 57443 +const DIV = 57444 +const MOD = 57445 +const POW = 57446 var yyToknames = [...]string{ "$end", @@ -226,6 +227,7 @@ var yyToknames = [...]string{ "COUNT_OVER_TIME", "MIN_OVER_TIME", "MAX_OVER_TIME", + "AVG_OVER_TIME", "QUANTILE_OVER_TIME", "HISTOGRAM_OVER_TIME", "COMPARE", @@ -272,166 +274,167 @@ var yyExca = [...]int{ -1, 1, 1, -1, -2, 0, - -1, 298, + -1, 300, 13, 86, -2, 94, } const yyPrivate = 57344 -const yyLast = 993 +const yyLast = 994 var yyAct = [...]int{ - 101, 5, 6, 8, 7, 100, 98, 12, 284, 18, - 247, 67, 90, 228, 77, 336, 13, 205, 348, 30, - 94, 99, 236, 237, 238, 247, 70, 29, 229, 296, - 2, 347, 153, 154, 157, 155, 329, 234, 235, 66, - 236, 237, 238, 247, 87, 88, 89, 90, 204, 328, - 185, 187, 188, 189, 190, 191, 192, 193, 194, 195, - 196, 197, 198, 199, 200, 201, 202, 333, 327, 324, - 323, 78, 79, 80, 81, 82, 83, 322, 211, 85, - 86, 321, 87, 88, 89, 90, 74, 75, 76, 77, - 383, 85, 86, 232, 87, 88, 89, 90, 231, 366, - 362, 219, 221, 222, 223, 224, 225, 226, 361, 345, - 332, 352, 227, 351, 230, 390, 250, 251, 252, 239, - 240, 241, 242, 243, 244, 246, 245, 355, 204, 248, - 249, 239, 240, 241, 242, 243, 244, 246, 245, 234, - 235, 256, 236, 237, 238, 247, 276, 274, 275, 393, - 303, 234, 235, 394, 236, 237, 238, 247, 356, 331, - 72, 73, 277, 74, 75, 76, 77, 293, 279, 280, - 281, 282, 248, 249, 239, 240, 241, 242, 243, 244, - 246, 245, 294, 354, 257, 258, 272, 209, 293, 205, - 389, 303, 387, 303, 234, 235, 353, 236, 237, 238, - 247, 273, 386, 303, 344, 262, 338, 153, 154, 157, - 155, 337, 263, 278, 264, 298, 378, 303, 209, 265, - 208, 248, 249, 239, 240, 241, 242, 243, 244, 246, - 245, 377, 303, 375, 376, 373, 372, 294, 357, 358, - 334, 335, 300, 234, 235, 388, 236, 237, 238, 247, - 302, 303, 17, 374, 186, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, - 319, 85, 86, 371, 87, 88, 89, 90, 370, 360, - 359, 295, 78, 79, 80, 81, 82, 83, 292, 232, - 232, 232, 232, 291, 231, 231, 231, 231, 290, 67, - 343, 67, 85, 86, 232, 87, 88, 89, 90, 231, - 230, 230, 230, 230, 70, 289, 70, 339, 340, 341, - 342, 288, 287, 286, 212, 230, 168, 151, 150, 300, - 149, 148, 346, 78, 79, 80, 81, 82, 83, 147, - 146, 92, 91, 350, 349, 17, 392, 153, 154, 157, - 155, 84, 385, 72, 73, 367, 74, 75, 76, 77, - 232, 232, 285, 71, 326, 231, 231, 143, 144, 145, - 325, 232, 232, 368, 369, 232, 231, 231, 382, 381, - 231, 230, 230, 261, 379, 380, 365, 364, 384, 232, - 260, 259, 230, 230, 231, 255, 230, 254, 253, 28, - 283, 363, 391, 69, 16, 102, 103, 104, 108, 131, - 230, 93, 95, 4, 152, 107, 105, 106, 110, 109, - 111, 112, 113, 114, 115, 116, 117, 118, 119, 120, - 121, 122, 124, 123, 125, 126, 10, 127, 128, 129, - 130, 156, 1, 0, 0, 0, 134, 132, 133, 138, - 139, 140, 135, 141, 136, 142, 137, 330, 0, 102, - 103, 104, 108, 131, 0, 0, 95, 0, 0, 107, - 105, 106, 110, 109, 111, 112, 113, 114, 115, 116, - 117, 118, 119, 120, 121, 122, 124, 123, 125, 126, - 0, 127, 128, 129, 130, 320, 68, 11, 96, 97, - 134, 132, 133, 138, 139, 140, 135, 141, 136, 142, - 137, 0, 0, 0, 0, 0, 0, 0, 0, 248, - 249, 239, 240, 241, 242, 243, 244, 246, 245, 301, - 0, 0, 0, 0, 0, 0, 0, 233, 0, 0, - 0, 234, 235, 0, 236, 237, 238, 247, 0, 0, - 0, 0, 96, 97, 0, 0, 0, 248, 249, 239, - 240, 241, 242, 243, 244, 246, 245, 0, 210, 213, - 214, 215, 216, 217, 218, 206, 0, 0, 0, 234, - 235, 0, 236, 237, 238, 247, 0, 0, 0, 0, - 0, 248, 249, 239, 240, 241, 242, 243, 244, 246, - 245, 248, 249, 239, 240, 241, 242, 243, 244, 246, - 245, 0, 0, 234, 235, 0, 236, 237, 238, 247, - 0, 0, 0, 234, 235, 0, 236, 237, 238, 247, - 19, 20, 21, 0, 17, 203, 165, 48, 53, 0, + 101, 5, 6, 8, 7, 98, 100, 285, 18, 12, + 248, 67, 90, 77, 338, 229, 206, 230, 205, 13, + 298, 2, 94, 30, 99, 237, 238, 239, 248, 70, + 66, 29, 153, 154, 157, 155, 235, 236, 205, 237, + 238, 239, 248, 85, 86, 390, 87, 88, 89, 90, + 186, 188, 189, 190, 191, 192, 193, 194, 195, 196, + 197, 198, 199, 200, 201, 202, 203, 335, 351, 350, + 371, 78, 79, 80, 81, 82, 83, 212, 72, 73, + 348, 74, 75, 76, 77, 87, 88, 89, 90, 331, + 367, 85, 86, 233, 87, 88, 89, 90, 330, 232, + 206, 329, 210, 326, 220, 222, 223, 224, 225, 226, + 227, 334, 402, 325, 228, 324, 323, 231, 251, 252, + 253, 240, 241, 242, 243, 244, 245, 247, 246, 366, + 249, 250, 240, 241, 242, 243, 244, 245, 247, 246, + 365, 235, 236, 355, 237, 238, 239, 248, 74, 75, + 76, 77, 235, 236, 333, 237, 238, 239, 248, 354, + 257, 398, 210, 275, 276, 401, 305, 359, 295, 277, + 280, 281, 282, 283, 249, 250, 240, 241, 242, 243, + 244, 245, 247, 246, 360, 278, 296, 85, 86, 295, + 87, 88, 89, 90, 208, 358, 235, 236, 273, 237, + 238, 239, 248, 258, 259, 397, 305, 357, 153, 154, + 157, 155, 356, 274, 347, 300, 340, 249, 250, 240, + 241, 242, 243, 244, 245, 247, 246, 78, 79, 80, + 81, 82, 83, 339, 302, 395, 305, 394, 305, 235, + 236, 296, 237, 238, 239, 248, 279, 85, 86, 209, + 87, 88, 89, 90, 393, 305, 384, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 320, 321, 383, 305, 381, 382, 396, 72, + 73, 380, 74, 75, 76, 77, 379, 378, 361, 362, + 233, 233, 233, 233, 233, 377, 232, 232, 232, 232, + 232, 67, 376, 67, 346, 375, 233, 341, 342, 343, + 344, 345, 232, 364, 231, 231, 231, 231, 231, 70, + 363, 70, 302, 349, 336, 337, 304, 305, 297, 17, + 231, 187, 294, 263, 78, 79, 80, 81, 82, 83, + 264, 293, 265, 292, 353, 352, 291, 266, 290, 289, + 153, 154, 157, 155, 72, 73, 288, 74, 75, 76, + 77, 287, 213, 169, 233, 233, 151, 150, 149, 148, + 232, 232, 147, 146, 92, 91, 233, 233, 233, 373, + 374, 233, 232, 232, 232, 17, 84, 232, 231, 231, + 400, 385, 386, 387, 389, 388, 391, 233, 71, 392, + 231, 231, 231, 232, 267, 231, 268, 270, 271, 372, + 269, 286, 399, 102, 103, 104, 108, 131, 272, 93, + 95, 231, 328, 107, 105, 106, 110, 109, 111, 112, + 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, + 124, 123, 125, 126, 327, 127, 128, 129, 130, 68, + 11, 143, 144, 145, 134, 132, 133, 138, 139, 140, + 135, 141, 136, 142, 137, 332, 370, 369, 102, 103, + 104, 108, 131, 262, 261, 95, 260, 256, 107, 105, + 106, 110, 109, 111, 112, 113, 114, 115, 116, 117, + 118, 119, 120, 121, 122, 124, 123, 125, 126, 255, + 127, 128, 129, 130, 322, 254, 28, 96, 97, 134, + 132, 133, 138, 139, 140, 135, 141, 136, 142, 137, + 284, 211, 214, 215, 216, 217, 218, 219, 249, 250, + 240, 241, 242, 243, 244, 245, 247, 246, 303, 368, + 69, 16, 4, 152, 10, 156, 234, 1, 0, 0, + 235, 236, 0, 237, 238, 239, 248, 0, 0, 0, + 0, 0, 96, 97, 0, 0, 0, 249, 250, 240, + 241, 242, 243, 244, 245, 247, 246, 0, 0, 0, + 0, 0, 0, 0, 207, 0, 0, 0, 0, 235, + 236, 0, 237, 238, 239, 248, 0, 0, 0, 0, + 0, 249, 250, 240, 241, 242, 243, 244, 245, 247, + 246, 249, 250, 240, 241, 242, 243, 244, 245, 247, + 246, 0, 0, 235, 236, 0, 237, 238, 239, 248, + 0, 0, 0, 235, 236, 0, 237, 238, 239, 248, + 19, 20, 21, 0, 17, 204, 166, 48, 53, 0, 0, 50, 0, 49, 0, 57, 0, 51, 52, 54, 55, 56, 59, 58, 60, 61, 64, 63, 62, 48, 53, 0, 0, 50, 0, 49, 0, 57, 0, 51, 52, 54, 55, 56, 59, 58, 60, 61, 64, 63, - 62, 23, 26, 24, 25, 27, 14, 166, 15, 0, - 158, 159, 160, 161, 162, 163, 164, 31, 36, 0, - 0, 33, 0, 32, 0, 42, 0, 34, 35, 37, - 38, 39, 40, 41, 43, 44, 45, 46, 47, 31, - 36, 0, 22, 33, 0, 32, 0, 42, 0, 34, - 35, 37, 38, 39, 40, 41, 43, 44, 45, 46, - 47, 19, 20, 21, 0, 17, 0, 165, 0, 19, - 20, 21, 0, 17, 0, 299, 0, 19, 20, 21, - 50, 17, 49, 297, 57, 0, 51, 52, 54, 55, - 56, 59, 58, 60, 61, 64, 63, 62, 0, 207, - 0, 0, 0, 0, 19, 20, 21, 0, 17, 0, - 9, 0, 23, 26, 24, 25, 27, 14, 166, 15, - 23, 26, 24, 25, 27, 14, 0, 15, 23, 26, - 24, 25, 27, 14, 0, 15, 0, 0, 0, 19, - 20, 21, 0, 17, 0, 165, 19, 20, 21, 0, - 0, 0, 220, 22, 0, 23, 26, 24, 25, 27, - 14, 22, 15, 0, 33, 0, 32, 0, 42, 22, + 62, 23, 26, 24, 25, 27, 14, 167, 15, 0, + 158, 159, 160, 161, 162, 163, 164, 165, 31, 36, + 0, 0, 33, 0, 32, 0, 42, 0, 34, 35, + 37, 38, 39, 40, 41, 43, 44, 45, 46, 47, + 31, 36, 0, 22, 33, 0, 32, 0, 42, 0, 34, 35, 37, 38, 39, 40, 41, 43, 44, 45, - 46, 47, 0, 72, 73, 0, 74, 75, 76, 77, - 23, 26, 24, 25, 27, 0, 22, 23, 26, 24, - 25, 27, 0, 266, 131, 267, 269, 270, 0, 268, - 0, 0, 0, 0, 0, 0, 0, 271, 0, 0, - 65, 3, 118, 119, 120, 121, 122, 124, 123, 125, - 126, 22, 127, 128, 129, 130, 0, 0, 22, 0, - 0, 134, 132, 133, 138, 139, 140, 135, 141, 136, - 142, 137, 167, 169, 170, 171, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 102, - 103, 104, 108, 0, 0, 0, 212, 0, 0, 107, - 105, 106, 110, 109, 111, 112, 113, 114, 115, 116, - 117, 102, 103, 104, 108, 0, 0, 0, 0, 0, - 0, 107, 105, 106, 110, 109, 111, 112, 113, 114, - 115, 116, 117, + 46, 47, 19, 20, 21, 0, 17, 0, 166, 0, + 19, 20, 21, 0, 17, 0, 301, 0, 19, 20, + 21, 50, 17, 49, 299, 57, 0, 51, 52, 54, + 55, 56, 59, 58, 60, 61, 64, 63, 62, 0, + 0, 0, 0, 0, 0, 0, 19, 20, 21, 0, + 17, 0, 9, 23, 26, 24, 25, 27, 14, 167, + 15, 23, 26, 24, 25, 27, 14, 0, 15, 23, + 26, 24, 25, 27, 14, 33, 15, 32, 0, 42, + 0, 34, 35, 37, 38, 39, 40, 41, 43, 44, + 45, 46, 47, 0, 0, 22, 0, 23, 26, 24, + 25, 27, 14, 22, 15, 131, 19, 20, 21, 0, + 17, 22, 166, 19, 20, 21, 0, 0, 0, 221, + 0, 0, 0, 118, 119, 120, 121, 122, 124, 123, + 125, 126, 0, 127, 128, 129, 130, 65, 3, 22, + 0, 0, 134, 132, 133, 138, 139, 140, 135, 141, + 136, 142, 137, 0, 0, 0, 0, 23, 26, 24, + 25, 27, 0, 0, 23, 26, 24, 25, 27, 168, + 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, + 180, 181, 182, 183, 184, 185, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 22, + 102, 103, 104, 108, 0, 0, 22, 213, 0, 0, + 107, 105, 106, 110, 109, 111, 112, 113, 114, 115, + 116, 117, 102, 103, 104, 108, 0, 0, 0, 0, + 0, 0, 107, 105, 106, 110, 109, 111, 112, 113, + 114, 115, 116, 117, } var yyPact = [...]int{ - 778, -46, -55, 644, -1000, 584, -1000, -1000, -1000, 778, - -1000, 256, -1000, -6, 330, 329, -1000, 400, -1000, -1000, - -1000, -1000, 361, 328, 327, 319, 318, 316, -1000, 315, - 624, 314, 314, 314, 314, 314, 314, 314, 314, 314, - 314, 314, 314, 314, 314, 314, 314, 314, 242, 242, - 242, 242, 242, 242, 242, 242, 242, 242, 242, 242, - 242, 242, 242, 242, 242, 622, 115, 562, 766, 207, - 205, 944, 312, 312, 312, 312, 312, 312, -1000, -1000, - -1000, -1000, -1000, -1000, 820, 820, 820, 820, 820, 820, - 820, 454, 875, -1000, 526, 454, 454, 454, -1000, -1000, + 790, -43, -52, 654, -1000, 593, -1000, -1000, -1000, 790, + -1000, 256, -1000, -7, 363, 362, -1000, 408, -1000, -1000, + -1000, -1000, 445, 361, 360, 357, 356, 355, -1000, 354, + 634, 351, 351, 351, 351, 351, 351, 351, 351, 351, + 351, 351, 351, 351, 351, 351, 351, 351, 319, 319, + 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, + 319, 319, 319, 319, 319, 632, 25, 571, 181, 236, + 149, 945, 350, 350, 350, 350, 350, 350, -1000, -1000, + -1000, -1000, -1000, -1000, 857, 857, 857, 857, 857, 857, + 857, 463, 846, -1000, 535, 463, 463, 463, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 394, 393, 391, 137, 387, 386, 379, 178, 856, - 157, 105, 117, -1000, -1000, -1000, 200, 454, 454, 454, - 454, 358, -1000, 584, -1000, -1000, -1000, -1000, 311, 310, - 309, 303, 286, 281, 276, 813, 269, 765, 751, -1000, - -1000, -1000, -1000, 765, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 681, 242, -1000, -1000, -1000, - -1000, 681, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 735, -1000, -1000, -1000, -1000, - 63, -1000, 743, -14, -14, -89, -89, -89, -89, -18, - 820, -56, -56, -91, -91, -91, -91, 516, 237, -1000, - -1000, -1000, -1000, -1000, 454, 454, 454, 454, 454, 454, - 454, 454, 454, 454, 454, 454, 454, 454, 454, 454, - 482, -78, -78, 16, 12, 5, 4, 366, 360, 3, - -16, -29, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 444, - 146, 97, 54, 227, -1000, -62, 198, 193, 875, 875, - 875, 875, 335, 562, 174, 191, 35, 751, -1000, 743, - -57, -1000, -1000, 875, -78, -78, -93, -93, -93, -60, - -60, -60, -60, -60, -60, -60, -60, -93, 42, 42, - -1000, -1000, -1000, -1000, -1000, -34, -47, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 358, 966, 51, 49, 183, - 170, 113, 145, 225, -1000, 735, -1000, -1000, -1000, -1000, - -1000, 268, 267, 46, 38, 380, 37, -1000, 349, 875, - 875, 266, 261, 222, -1000, -1000, 241, 220, 218, 203, - 875, 875, 372, 28, 875, -1000, 346, -1000, -1000, 189, - 179, -1000, -1000, 233, 177, 101, -1000, -1000, 875, -1000, - 340, 136, 140, -1000, -1000, + -1000, 501, 495, 473, 156, 472, 470, 469, 306, 377, + 169, 121, 140, -1000, -1000, -1000, 233, 463, 463, 463, + 463, 407, -1000, 593, -1000, -1000, -1000, -1000, 349, 344, + 337, 336, 334, 331, 329, 320, 850, 316, 745, 762, + -1000, -1000, -1000, -1000, 745, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 691, 319, -1000, -1000, + -1000, -1000, 691, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 746, -1000, -1000, -1000, + -1000, -20, -1000, 754, 47, 47, -91, -91, -91, -91, + -55, 857, -16, -16, -92, -92, -92, -92, 525, 313, + -1000, -1000, -1000, -1000, -1000, 463, 463, 463, 463, 463, + 463, 463, 463, 463, 463, 463, 463, 463, 463, 463, + 463, 491, -76, -76, 51, 50, 48, 38, 440, 418, + 36, 33, 24, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 452, 141, 98, 54, 311, -1000, -64, 220, 203, 846, + 846, 846, 846, 846, 375, 571, 89, 201, 5, 762, + -1000, 754, -59, -1000, -1000, 846, -76, -76, -94, -94, + -94, -62, -62, -62, -62, -62, -62, -62, -62, -94, + 43, 43, -1000, -1000, -1000, -1000, -1000, 4, 3, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 407, 967, 97, + 81, 199, 194, 182, 153, 171, 275, -1000, 746, -1000, + -1000, -1000, -1000, -1000, 308, 301, 78, 67, 28, 460, + 8, -1000, 403, 846, 846, 293, 290, 283, 273, -1000, + -1000, 269, 263, 261, 243, 846, 846, 846, 388, -17, + 846, -1000, 393, -1000, -1000, 241, 224, 222, -1000, -1000, + 266, 192, 147, -1000, -1000, -1000, 846, -1000, 384, 152, + 99, -1000, -1000, } var yyPgo = [...]int{ - 0, 442, 4, 441, 3, 13, 1, 900, 436, 29, - 7, 2, 351, 414, 413, 496, 16, 404, 403, 9, - 20, 6, 21, 5, 0, 28, 401, 8, 400, 399, + 0, 547, 4, 545, 3, 15, 1, 887, 544, 20, + 9, 2, 386, 543, 542, 449, 19, 541, 540, 8, + 22, 5, 24, 6, 0, 17, 539, 7, 520, 506, } var yyR1 = [...]int{ @@ -447,15 +450,15 @@ var yyR1 = [...]int{ 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 19, 19, 19, 19, 19, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, - 13, 27, 29, 28, 28, 20, 20, 20, 20, 20, + 13, 13, 13, 27, 29, 28, 28, 20, 20, 20, + 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, - 20, 20, 20, 20, 20, 20, 20, 20, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, - 21, 21, 21, 21, 22, 22, 22, 22, 22, 22, - 22, 22, 22, 22, 22, 22, 22, 24, 24, 24, + 21, 21, 21, 21, 21, 21, 22, 22, 22, 22, + 22, 22, 22, 22, 22, 22, 22, 22, 22, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, - 24, 24, 24, 23, 23, 23, 23, 23, 23, 23, - 23, 23, + 24, 24, 24, 24, 24, 23, 23, 23, 23, 23, + 23, 23, 23, 23, } var yyR2 = [...]int{ @@ -470,59 +473,60 @@ var yyR2 = [...]int{ 3, 3, 3, 3, 1, 3, 3, 3, 3, 3, 3, 3, 3, 3, 1, 1, 1, 1, 2, 2, 2, 3, 4, 4, 4, 4, 3, 7, 3, 7, - 4, 8, 4, 8, 6, 10, 4, 8, 4, 6, - 10, 3, 4, 1, 3, 3, 3, 3, 3, 3, + 4, 8, 4, 8, 4, 8, 6, 10, 4, 8, + 4, 6, 10, 3, 4, 1, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 2, 2, 1, 1, 1, 1, 1, 1, + 3, 3, 3, 3, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 2, 2, 2, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 3, 3, 3, 3, 4, 4, 3, - 3, 3, + 2, 2, 2, 2, 2, 3, 3, 3, 3, 4, + 4, 3, 3, 3, } var yyChk = [...]int{ -1000, -1, -9, -7, -14, -6, -11, -2, -4, 12, -8, -15, -10, -16, 62, 64, -17, 10, -19, 6, - 7, 8, 98, 57, 59, 60, 58, 61, -29, 73, - 74, 75, 81, 79, 85, 86, 76, 87, 88, 89, - 90, 91, 83, 92, 93, 94, 95, 96, 75, 81, - 79, 85, 86, 76, 87, 88, 89, 83, 91, 90, - 92, 93, 96, 95, 94, -7, -9, -6, -15, -18, - -16, -12, 97, 98, 100, 101, 102, 103, 77, 78, - 79, 80, 81, 82, -12, 97, 98, 100, 101, 102, - 103, 12, 12, 11, -20, 12, 98, 99, -21, -22, + 7, 8, 99, 57, 59, 60, 58, 61, -29, 74, + 75, 76, 82, 80, 86, 87, 77, 88, 89, 90, + 91, 92, 84, 93, 94, 95, 96, 97, 76, 82, + 80, 86, 87, 77, 88, 89, 90, 84, 92, 91, + 93, 94, 97, 96, 95, -7, -9, -6, -15, -18, + -16, -12, 98, 99, 101, 102, 103, 104, 78, 79, + 80, 81, 82, 83, -12, 98, 99, 101, 102, 103, + 104, 12, 12, 11, -20, 12, 99, 100, -21, -22, -23, -24, 5, 6, 7, 16, 17, 15, 8, 19, 18, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 33, 32, 34, 35, 37, 38, 39, 40, 9, 47, 48, 46, 52, 54, 56, 49, 50, 51, 53, 55, 6, 7, 8, 12, 12, 12, 12, 12, 12, -13, -6, -11, -2, -3, -4, 66, 67, - 68, 69, 70, 71, 72, 12, 63, -7, 12, -7, + 68, 69, 70, 71, 72, 73, 12, 63, -7, 12, -7, -7, -7, -7, -7, -7, -7, -7, -7, -7, - -7, -7, -7, -7, -7, -6, 12, -6, -6, -6, + -7, -7, -7, -7, -7, -7, -6, 12, -6, -6, -6, -6, -6, -6, -6, -6, -6, -6, -6, -6, - -6, -6, -6, 13, 13, 74, 13, 13, 13, 13, - -15, -21, 12, -15, -15, -15, -15, -15, -15, -16, - 12, -16, -16, -16, -16, -16, -16, -20, -5, -25, - -22, -23, -24, 11, 97, 98, 100, 101, 102, 77, - 78, 79, 80, 81, 82, 84, 83, 103, 75, 76, - -20, -20, -20, 4, 4, 4, 4, 47, 48, 4, - 4, 4, 27, 34, 36, 41, 27, 29, 33, 30, - 31, 41, 29, 44, 42, 43, 29, 45, 13, -20, - -20, -20, -20, -28, -27, 4, 12, 12, 12, 12, - 12, 12, 12, -6, -16, 12, -9, 12, -19, 12, - -9, 13, 13, 14, -20, -20, -20, -20, -20, -20, + -6, -6, -6, -6, 13, 13, 75, 13, 13, 13, + 13, -15, -21, 12, -15, -15, -15, -15, -15, -15, + -16, 12, -16, -16, -16, -16, -16, -16, -20, -5, + -25, -22, -23, -24, 11, 98, 99, 101, 102, 103, + 78, 79, 80, 81, 82, 83, 85, 84, 104, 76, + 77, -20, -20, -20, 4, 4, 4, 4, 47, 48, + 4, 4, 4, 27, 34, 36, 41, 27, 29, 33, + 30, 31, 41, 29, 44, 42, 43, 29, 45, 13, + -20, -20, -20, -20, -28, -27, 4, 12, 12, 12, + 12, 12, 12, 12, 12, -6, -16, 12, -9, 12, + -19, 12, -9, 13, 13, 14, -20, -20, -20, -20, -20, -20, -20, -20, -20, -20, -20, -20, -20, -20, - 13, 65, 65, 65, 65, 4, 4, 65, 65, 65, - 13, 13, 13, 13, 13, 14, 77, 13, 13, -25, - -25, -25, -25, -10, 13, 74, -25, 65, 65, -27, - -21, 62, 62, 13, 13, 14, 13, 13, 14, 12, - 12, 62, 62, -26, 7, 6, 62, 6, -5, -5, - 12, 12, 14, 13, 12, 13, 14, 13, 13, -5, - -5, 7, 6, 62, -5, 6, 13, 13, 12, 13, - 14, -5, 6, 13, 13, + -20, -20, 13, 65, 65, 65, 65, 4, 4, 65, + 65, 65, 13, 13, 13, 13, 13, 14, 78, 13, + 13, -25, -25, -25, -25, -25, -10, 13, 75, -25, + 65, 65, -27, -21, 62, 62, 13, 13, 13, 14, + 13, 13, 14, 12, 12, 62, 62, 62, -26, 7, + 6, 62, 6, -5, -5, 12, 12, 12, 14, 13, + 12, 13, 14, 13, 13, -5, -5, -5, 7, 6, + 62, -5, 6, 13, 13, 13, 12, 13, 14, -5, + 6, 13, 13, } var yyDef = [...]int{ @@ -535,37 +539,38 @@ var yyDef = [...]int{ 0, 0, 0, 0, 0, 0, 0, 26, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 69, 70, 71, 72, 73, 74, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 66, 0, 0, 0, 0, 144, 145, - 146, 147, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, 159, 160, 161, 162, 163, 164, 165, - 166, 167, 168, 169, 170, 171, 172, 173, 174, 175, - 176, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 66, 0, 0, 0, 0, 146, 147, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, 159, 160, 161, 162, 163, 164, 165, 166, 167, + 168, 169, 170, 171, 172, 173, 174, 175, 176, 177, + 178, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 98, 99, 100, 0, 0, 0, 0, 0, 0, 4, 30, 31, 32, 33, 34, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 7, 0, 8, - 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, - 19, 20, 21, 22, 23, 48, 0, 49, 50, 51, - 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, - 62, 63, 64, 6, 25, 0, 47, 77, 85, 87, - 75, 76, 0, 78, 79, 80, 81, 82, 83, 68, - 0, 88, 89, 90, 91, 92, 93, 0, 0, 41, - 38, 39, 40, 67, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 7, 0, + 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, + 18, 19, 20, 21, 22, 23, 48, 0, 49, 50, + 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, + 61, 62, 63, 64, 6, 25, 0, 47, 77, 85, + 87, 75, 76, 0, 78, 79, 80, 81, 82, 83, + 68, 0, 88, 89, 90, 91, 92, 93, 0, 0, + 41, 38, 39, 40, 67, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 144, 145, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 179, 180, 181, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, 194, 101, + 0, 0, 0, 0, 0, 125, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 142, 143, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 177, 178, 179, 180, 181, 182, 183, 184, - 185, 186, 187, 188, 189, 190, 191, 192, 101, 0, - 0, 0, 0, 0, 123, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, -2, 0, - 0, 35, 37, 0, 126, 127, 128, 129, 130, 131, + -2, 0, 0, 35, 37, 0, 128, 129, 130, 131, 132, 133, 134, 135, 136, 137, 138, 139, 140, 141, - 125, 193, 194, 195, 196, 0, 0, 199, 200, 201, - 102, 103, 104, 105, 122, 0, 0, 106, 108, 0, - 0, 0, 0, 0, 36, 0, 42, 197, 198, 124, - 121, 0, 0, 110, 112, 0, 116, 118, 0, 0, - 0, 0, 0, 0, 43, 44, 0, 0, 0, 0, - 0, 0, 0, 114, 0, 119, 0, 107, 109, 0, - 0, 45, 46, 0, 0, 0, 111, 113, 0, 117, - 0, 0, 0, 115, 120, + 142, 143, 127, 195, 196, 197, 198, 0, 0, 201, + 202, 203, 102, 103, 104, 105, 124, 0, 0, 106, + 108, 0, 0, 0, 0, 0, 0, 36, 0, 42, + 199, 200, 126, 123, 0, 0, 110, 112, 114, 0, + 118, 120, 0, 0, 0, 0, 0, 0, 0, 43, + 44, 0, 0, 0, 0, 0, 0, 0, 0, 116, + 0, 121, 0, 107, 109, 0, 0, 0, 45, 46, + 0, 0, 0, 111, 113, 115, 0, 119, 0, 0, + 0, 117, 122, } var yyTok1 = [...]int{ @@ -583,7 +588,7 @@ var yyTok2 = [...]int{ 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, - 102, 103, + 102, 103, 104, } var yyTok3 = [...]int{ 0, @@ -1605,530 +1610,542 @@ yydefault: yyVAL.metricsAggregation = newMetricsAggregateWithAttr(metricsAggregateMaxOverTime, yyDollar[3].attribute, yyDollar[7].attributeList) } case 114: - yyDollar = yyS[yypt-6 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line pkg/traceql/expr.y:305 { - yyVAL.metricsAggregation = newMetricsAggregateQuantileOverTime(yyDollar[3].attribute, yyDollar[5].numericList, nil) + yyVAL.metricsAggregation = newAverageOverTimeMetricsAggregator(yyDollar[3].attribute, nil) } case 115: - yyDollar = yyS[yypt-10 : yypt+1] + yyDollar = yyS[yypt-8 : yypt+1] //line pkg/traceql/expr.y:306 { - yyVAL.metricsAggregation = newMetricsAggregateQuantileOverTime(yyDollar[3].attribute, yyDollar[5].numericList, yyDollar[9].attributeList) + yyVAL.metricsAggregation = newAverageOverTimeMetricsAggregator(yyDollar[3].attribute, yyDollar[7].attributeList) } case 116: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] //line pkg/traceql/expr.y:307 { - yyVAL.metricsAggregation = newMetricsAggregateWithAttr(metricsAggregateHistogramOverTime, yyDollar[3].attribute, nil) + yyVAL.metricsAggregation = newMetricsAggregateQuantileOverTime(yyDollar[3].attribute, yyDollar[5].numericList, nil) } case 117: - yyDollar = yyS[yypt-8 : yypt+1] + yyDollar = yyS[yypt-10 : yypt+1] //line pkg/traceql/expr.y:308 { - yyVAL.metricsAggregation = newMetricsAggregateWithAttr(metricsAggregateHistogramOverTime, yyDollar[3].attribute, yyDollar[7].attributeList) + yyVAL.metricsAggregation = newMetricsAggregateQuantileOverTime(yyDollar[3].attribute, yyDollar[5].numericList, yyDollar[9].attributeList) } case 118: yyDollar = yyS[yypt-4 : yypt+1] //line pkg/traceql/expr.y:309 { - yyVAL.metricsAggregation = newMetricsCompare(yyDollar[3].spansetFilter, 10, 0, 0) + yyVAL.metricsAggregation = newMetricsAggregateWithAttr(metricsAggregateHistogramOverTime, yyDollar[3].attribute, nil) } case 119: - yyDollar = yyS[yypt-6 : yypt+1] + yyDollar = yyS[yypt-8 : yypt+1] //line pkg/traceql/expr.y:310 { - yyVAL.metricsAggregation = newMetricsCompare(yyDollar[3].spansetFilter, yyDollar[5].staticInt, 0, 0) + yyVAL.metricsAggregation = newMetricsAggregateWithAttr(metricsAggregateHistogramOverTime, yyDollar[3].attribute, yyDollar[7].attributeList) } case 120: - yyDollar = yyS[yypt-10 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line pkg/traceql/expr.y:311 { - yyVAL.metricsAggregation = newMetricsCompare(yyDollar[3].spansetFilter, yyDollar[5].staticInt, yyDollar[7].staticInt, yyDollar[9].staticInt) + yyVAL.metricsAggregation = newMetricsCompare(yyDollar[3].spansetFilter, 10, 0, 0) } case 121: - yyDollar = yyS[yypt-3 : yypt+1] -//line pkg/traceql/expr.y:318 + yyDollar = yyS[yypt-6 : yypt+1] +//line pkg/traceql/expr.y:312 { - yyVAL.hint = newHint(yyDollar[1].staticStr, yyDollar[3].static) + yyVAL.metricsAggregation = newMetricsCompare(yyDollar[3].spansetFilter, yyDollar[5].staticInt, 0, 0) } case 122: - yyDollar = yyS[yypt-4 : yypt+1] -//line pkg/traceql/expr.y:322 + yyDollar = yyS[yypt-10 : yypt+1] +//line pkg/traceql/expr.y:313 { - yyVAL.hints = newHints(yyDollar[3].hintList) + yyVAL.metricsAggregation = newMetricsCompare(yyDollar[3].spansetFilter, yyDollar[5].staticInt, yyDollar[7].staticInt, yyDollar[9].staticInt) } case 123: - yyDollar = yyS[yypt-1 : yypt+1] -//line pkg/traceql/expr.y:326 + yyDollar = yyS[yypt-3 : yypt+1] +//line pkg/traceql/expr.y:320 { - yyVAL.hintList = []*Hint{yyDollar[1].hint} + yyVAL.hint = newHint(yyDollar[1].staticStr, yyDollar[3].static) } case 124: - yyDollar = yyS[yypt-3 : yypt+1] -//line pkg/traceql/expr.y:327 + yyDollar = yyS[yypt-4 : yypt+1] +//line pkg/traceql/expr.y:324 { - yyVAL.hintList = append(yyDollar[1].hintList, yyDollar[3].hint) + yyVAL.hints = newHints(yyDollar[3].hintList) } case 125: - yyDollar = yyS[yypt-3 : yypt+1] -//line pkg/traceql/expr.y:335 + yyDollar = yyS[yypt-1 : yypt+1] +//line pkg/traceql/expr.y:328 { - yyVAL.fieldExpression = yyDollar[2].fieldExpression + yyVAL.hintList = []*Hint{yyDollar[1].hint} } case 126: yyDollar = yyS[yypt-3 : yypt+1] -//line pkg/traceql/expr.y:336 +//line pkg/traceql/expr.y:329 { - yyVAL.fieldExpression = newBinaryOperation(OpAdd, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.hintList = append(yyDollar[1].hintList, yyDollar[3].hint) } case 127: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:337 { - yyVAL.fieldExpression = newBinaryOperation(OpSub, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = yyDollar[2].fieldExpression } case 128: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:338 { - yyVAL.fieldExpression = newBinaryOperation(OpMult, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpAdd, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 129: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:339 { - yyVAL.fieldExpression = newBinaryOperation(OpDiv, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpSub, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 130: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:340 { - yyVAL.fieldExpression = newBinaryOperation(OpMod, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpMult, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 131: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:341 { - yyVAL.fieldExpression = newBinaryOperation(OpEqual, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpDiv, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 132: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:342 { - yyVAL.fieldExpression = newBinaryOperation(OpNotEqual, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpMod, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 133: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:343 { - yyVAL.fieldExpression = newBinaryOperation(OpLess, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpEqual, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 134: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:344 { - yyVAL.fieldExpression = newBinaryOperation(OpLessEqual, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpNotEqual, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 135: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:345 { - yyVAL.fieldExpression = newBinaryOperation(OpGreater, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpLess, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 136: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:346 { - yyVAL.fieldExpression = newBinaryOperation(OpGreaterEqual, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpLessEqual, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 137: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:347 { - yyVAL.fieldExpression = newBinaryOperation(OpRegex, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpGreater, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 138: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:348 { - yyVAL.fieldExpression = newBinaryOperation(OpNotRegex, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpGreaterEqual, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 139: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:349 { - yyVAL.fieldExpression = newBinaryOperation(OpPower, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpRegex, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 140: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:350 { - yyVAL.fieldExpression = newBinaryOperation(OpAnd, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpNotRegex, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 141: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:351 { - yyVAL.fieldExpression = newBinaryOperation(OpOr, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpPower, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 142: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:352 { - yyVAL.fieldExpression = newUnaryOperation(OpSub, yyDollar[2].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpAnd, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 143: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:353 { - yyVAL.fieldExpression = newUnaryOperation(OpNot, yyDollar[2].fieldExpression) + yyVAL.fieldExpression = newBinaryOperation(OpOr, yyDollar[1].fieldExpression, yyDollar[3].fieldExpression) } case 144: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line pkg/traceql/expr.y:354 { - yyVAL.fieldExpression = yyDollar[1].static + yyVAL.fieldExpression = newUnaryOperation(OpSub, yyDollar[2].fieldExpression) } case 145: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line pkg/traceql/expr.y:355 { - yyVAL.fieldExpression = yyDollar[1].intrinsicField + yyVAL.fieldExpression = newUnaryOperation(OpNot, yyDollar[2].fieldExpression) } case 146: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:356 { - yyVAL.fieldExpression = yyDollar[1].attributeField + yyVAL.fieldExpression = yyDollar[1].static } case 147: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:357 { - yyVAL.fieldExpression = yyDollar[1].scopedIntrinsicField + yyVAL.fieldExpression = yyDollar[1].intrinsicField } case 148: yyDollar = yyS[yypt-1 : yypt+1] -//line pkg/traceql/expr.y:364 +//line pkg/traceql/expr.y:358 { - yyVAL.static = NewStaticString(yyDollar[1].staticStr) + yyVAL.fieldExpression = yyDollar[1].attributeField } case 149: yyDollar = yyS[yypt-1 : yypt+1] -//line pkg/traceql/expr.y:365 +//line pkg/traceql/expr.y:359 { - yyVAL.static = NewStaticInt(yyDollar[1].staticInt) + yyVAL.fieldExpression = yyDollar[1].scopedIntrinsicField } case 150: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:366 { - yyVAL.static = NewStaticFloat(yyDollar[1].staticFloat) + yyVAL.static = NewStaticString(yyDollar[1].staticStr) } case 151: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:367 { - yyVAL.static = NewStaticBool(true) + yyVAL.static = NewStaticInt(yyDollar[1].staticInt) } case 152: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:368 { - yyVAL.static = NewStaticBool(false) + yyVAL.static = NewStaticFloat(yyDollar[1].staticFloat) } case 153: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:369 { - yyVAL.static = NewStaticNil() + yyVAL.static = NewStaticBool(true) } case 154: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:370 { - yyVAL.static = NewStaticDuration(yyDollar[1].staticDuration) + yyVAL.static = NewStaticBool(false) } case 155: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:371 { - yyVAL.static = NewStaticStatus(StatusOk) + yyVAL.static = NewStaticNil() } case 156: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:372 { - yyVAL.static = NewStaticStatus(StatusError) + yyVAL.static = NewStaticDuration(yyDollar[1].staticDuration) } case 157: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:373 { - yyVAL.static = NewStaticStatus(StatusUnset) + yyVAL.static = NewStaticStatus(StatusOk) } case 158: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:374 { - yyVAL.static = NewStaticKind(KindUnspecified) + yyVAL.static = NewStaticStatus(StatusError) } case 159: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:375 { - yyVAL.static = NewStaticKind(KindInternal) + yyVAL.static = NewStaticStatus(StatusUnset) } case 160: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:376 { - yyVAL.static = NewStaticKind(KindServer) + yyVAL.static = NewStaticKind(KindUnspecified) } case 161: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:377 { - yyVAL.static = NewStaticKind(KindClient) + yyVAL.static = NewStaticKind(KindInternal) } case 162: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:378 { - yyVAL.static = NewStaticKind(KindProducer) + yyVAL.static = NewStaticKind(KindServer) } case 163: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:379 { - yyVAL.static = NewStaticKind(KindConsumer) + yyVAL.static = NewStaticKind(KindClient) } case 164: yyDollar = yyS[yypt-1 : yypt+1] -//line pkg/traceql/expr.y:385 +//line pkg/traceql/expr.y:380 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicDuration) + yyVAL.static = NewStaticKind(KindProducer) } case 165: yyDollar = yyS[yypt-1 : yypt+1] -//line pkg/traceql/expr.y:386 +//line pkg/traceql/expr.y:381 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicChildCount) + yyVAL.static = NewStaticKind(KindConsumer) } case 166: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:387 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicName) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicDuration) } case 167: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:388 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicStatus) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicChildCount) } case 168: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:389 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicStatusMessage) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicName) } case 169: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:390 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicKind) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicStatus) } case 170: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:391 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicParent) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicStatusMessage) } case 171: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:392 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicTraceRootSpan) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicKind) } case 172: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:393 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicTraceRootService) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicParent) } case 173: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:394 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicTraceDuration) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicTraceRootSpan) } case 174: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:395 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicNestedSetLeft) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicTraceRootService) } case 175: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:396 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicNestedSetRight) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicTraceDuration) } case 176: yyDollar = yyS[yypt-1 : yypt+1] //line pkg/traceql/expr.y:397 { - yyVAL.intrinsicField = NewIntrinsic(IntrinsicNestedSetParent) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicNestedSetLeft) } case 177: - yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:402 + yyDollar = yyS[yypt-1 : yypt+1] +//line pkg/traceql/expr.y:398 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicTraceDuration) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicNestedSetRight) } case 178: - yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:403 + yyDollar = yyS[yypt-1 : yypt+1] +//line pkg/traceql/expr.y:399 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicTraceRootSpan) + yyVAL.intrinsicField = NewIntrinsic(IntrinsicNestedSetParent) } case 179: yyDollar = yyS[yypt-2 : yypt+1] //line pkg/traceql/expr.y:404 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicTraceRootService) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicTraceDuration) } case 180: yyDollar = yyS[yypt-2 : yypt+1] //line pkg/traceql/expr.y:405 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicTraceID) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicTraceRootSpan) } case 181: yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:407 +//line pkg/traceql/expr.y:406 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicDuration) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicTraceRootService) } case 182: yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:408 +//line pkg/traceql/expr.y:407 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicName) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicTraceID) } case 183: yyDollar = yyS[yypt-2 : yypt+1] //line pkg/traceql/expr.y:409 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicKind) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicDuration) } case 184: yyDollar = yyS[yypt-2 : yypt+1] //line pkg/traceql/expr.y:410 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicStatus) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicName) } case 185: yyDollar = yyS[yypt-2 : yypt+1] //line pkg/traceql/expr.y:411 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicStatusMessage) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicKind) } case 186: yyDollar = yyS[yypt-2 : yypt+1] //line pkg/traceql/expr.y:412 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicSpanID) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicStatus) } case 187: yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:414 +//line pkg/traceql/expr.y:413 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicEventName) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicStatusMessage) } case 188: yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:415 +//line pkg/traceql/expr.y:414 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicEventTimeSinceStart) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicSpanID) } case 189: yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:417 +//line pkg/traceql/expr.y:416 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicLinkTraceID) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicEventName) } case 190: yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:418 +//line pkg/traceql/expr.y:417 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicLinkSpanID) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicEventTimeSinceStart) } case 191: yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:420 +//line pkg/traceql/expr.y:419 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicInstrumentationName) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicLinkTraceID) } case 192: yyDollar = yyS[yypt-2 : yypt+1] -//line pkg/traceql/expr.y:421 +//line pkg/traceql/expr.y:420 { - yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicInstrumentationVersion) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicLinkSpanID) } case 193: - yyDollar = yyS[yypt-3 : yypt+1] -//line pkg/traceql/expr.y:425 + yyDollar = yyS[yypt-2 : yypt+1] +//line pkg/traceql/expr.y:422 { - yyVAL.attributeField = NewAttribute(yyDollar[2].staticStr) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicInstrumentationName) } case 194: - yyDollar = yyS[yypt-3 : yypt+1] -//line pkg/traceql/expr.y:426 + yyDollar = yyS[yypt-2 : yypt+1] +//line pkg/traceql/expr.y:423 { - yyVAL.attributeField = NewScopedAttribute(AttributeScopeResource, false, yyDollar[2].staticStr) + yyVAL.scopedIntrinsicField = NewIntrinsic(IntrinsicInstrumentationVersion) } case 195: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:427 { - yyVAL.attributeField = NewScopedAttribute(AttributeScopeSpan, false, yyDollar[2].staticStr) + yyVAL.attributeField = NewAttribute(yyDollar[2].staticStr) } case 196: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:428 { - yyVAL.attributeField = NewScopedAttribute(AttributeScopeNone, true, yyDollar[2].staticStr) + yyVAL.attributeField = NewScopedAttribute(AttributeScopeResource, false, yyDollar[2].staticStr) } case 197: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:429 { - yyVAL.attributeField = NewScopedAttribute(AttributeScopeResource, true, yyDollar[3].staticStr) + yyVAL.attributeField = NewScopedAttribute(AttributeScopeSpan, false, yyDollar[2].staticStr) } case 198: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:430 { - yyVAL.attributeField = NewScopedAttribute(AttributeScopeSpan, true, yyDollar[3].staticStr) + yyVAL.attributeField = NewScopedAttribute(AttributeScopeNone, true, yyDollar[2].staticStr) } case 199: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line pkg/traceql/expr.y:431 { - yyVAL.attributeField = NewScopedAttribute(AttributeScopeEvent, false, yyDollar[2].staticStr) + yyVAL.attributeField = NewScopedAttribute(AttributeScopeResource, true, yyDollar[3].staticStr) } case 200: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line pkg/traceql/expr.y:432 { - yyVAL.attributeField = NewScopedAttribute(AttributeScopeLink, false, yyDollar[2].staticStr) + yyVAL.attributeField = NewScopedAttribute(AttributeScopeSpan, true, yyDollar[3].staticStr) } case 201: yyDollar = yyS[yypt-3 : yypt+1] //line pkg/traceql/expr.y:433 + { + yyVAL.attributeField = NewScopedAttribute(AttributeScopeEvent, false, yyDollar[2].staticStr) + } + case 202: + yyDollar = yyS[yypt-3 : yypt+1] +//line pkg/traceql/expr.y:434 + { + yyVAL.attributeField = NewScopedAttribute(AttributeScopeLink, false, yyDollar[2].staticStr) + } + case 203: + yyDollar = yyS[yypt-3 : yypt+1] +//line pkg/traceql/expr.y:435 { yyVAL.attributeField = NewScopedAttribute(AttributeScopeInstrumentation, false, yyDollar[2].staticStr) } diff --git a/pkg/traceql/lexer.go b/pkg/traceql/lexer.go index 1960c9c9551..2306b34ca71 100644 --- a/pkg/traceql/lexer.go +++ b/pkg/traceql/lexer.go @@ -104,6 +104,7 @@ var tokens = map[string]int{ "count_over_time": COUNT_OVER_TIME, "min_over_time": MIN_OVER_TIME, "max_over_time": MAX_OVER_TIME, + "avg_over_time": AVG_OVER_TIME, "quantile_over_time": QUANTILE_OVER_TIME, "histogram_over_time": HISTOGRAM_OVER_TIME, "compare": COMPARE, diff --git a/pkg/traceql/parse_test.go b/pkg/traceql/parse_test.go index c23101858a0..fe24606e00a 100644 --- a/pkg/traceql/parse_test.go +++ b/pkg/traceql/parse_test.go @@ -1393,6 +1393,18 @@ func TestMetrics(t *testing.T) { }), ), }, + { + in: `{ } | avg_over_time(duration) by(name, span.http.status_code)`, + expected: newRootExprWithMetrics( + newPipeline(newSpansetFilter(NewStaticBool(true))), + newAverageOverTimeMetricsAggregator( + NewIntrinsic(IntrinsicDuration), + []Attribute{ + NewIntrinsic(IntrinsicName), + NewScopedAttribute(AttributeScopeSpan, false, "http.status_code"), + }), + ), + }, { in: `{ } | quantile_over_time(duration, 0, 0.90, 0.95, 1) by(name, span.http.status_code)`, expected: newRootExprWithMetrics( diff --git a/pkg/traceql/test_examples.yaml b/pkg/traceql/test_examples.yaml index 69c7da8dcf0..836fbaa3dbe 100644 --- a/pkg/traceql/test_examples.yaml +++ b/pkg/traceql/test_examples.yaml @@ -145,6 +145,8 @@ valid: - '{} | rate()' - '{} | count_over_time() by (name) with(sample=0.1)' - '{} | min_over_time(duration) by (span.http.path)' + - '{} | max_over_time(duration) by (span.http.path)' + - '{} | avg_over_time(duration) by (span.http.path)' - '{} | quantile_over_time(duration, 0, 0.9, 1) by (span.http.path)' # undocumented - nested set - '{ nestedSetLeft > 3 }'