diff --git a/pkg/engine/compat.go b/pkg/engine/compat.go index 8aee55d63ba33..ca052dac34153 100644 --- a/pkg/engine/compat.go +++ b/pkg/engine/compat.go @@ -126,7 +126,8 @@ func (b *streamsResultBuilder) CollectRecord(rec arrow.RecordBatch) { }) // One of the parsed columns - case ident.ColumnType() == types.ColumnTypeParsed: + case ident.ColumnType() == types.ColumnTypeParsed || (ident.ColumnType() == types.ColumnTypeGenerated && + shortName == types.ColumnNameError || shortName == types.ColumnNameErrorDetails): parsedCol := col.(*array.String) // TODO: keep errors if --strict is set diff --git a/pkg/engine/internal/executor/aggregator.go b/pkg/engine/internal/executor/aggregator.go index 28d54a3751f18..c0ff97725bf88 100644 --- a/pkg/engine/internal/executor/aggregator.go +++ b/pkg/engine/internal/executor/aggregator.go @@ -1,7 +1,6 @@ package executor import ( - "fmt" "maps" "slices" "strings" @@ -12,14 +11,14 @@ import ( "github.com/apache/arrow-go/v18/arrow/memory" "github.com/cespare/xxhash/v2" - "github.com/grafana/loki/v3/pkg/engine/internal/planner/physical" "github.com/grafana/loki/v3/pkg/engine/internal/semconv" - "github.com/grafana/loki/v3/pkg/engine/internal/types" ) type groupState struct { - value float64 // aggregated value - labelValues []string // grouping label values + value float64 // aggregated value + count int64 // values counter + labels []arrow.Field // grouping labels + labelValues []string // grouping label values } type aggregationOperation int @@ -29,23 +28,23 @@ const ( aggregationOperationMax aggregationOperationMin aggregationOperationCount + aggregationOperationAvg + aggregationOperationStddev + aggregationOperationStdvar + aggregationOperationBytes ) // aggregator is used to aggregate sample values by a set of grouping keys for each point in time. type aggregator struct { - groupBy []physical.ColumnExpression // columns to group by points map[time.Time]map[uint64]*groupState // holds the groupState for each point in time series digest *xxhash.Digest // used to compute key for each group operation aggregationOperation // aggregation type + labels []arrow.Field // combined list of all label fields for all sample values } -// newAggregator creates a new aggregator with the specified groupBy columns. -// empty groupBy indicates no grouping. All values are aggregated into a single group. -// TODO: add without argument to support `without(...)` grouping. -// A special case of `without()` that has empty groupBy is used for Noop grouping which retains the input labels as is. -func newAggregator(groupBy []physical.ColumnExpression, pointsSizeHint int, operation aggregationOperation) *aggregator { +// newAggregator creates a new aggregator with the specified grouping. +func newAggregator(pointsSizeHint int, operation aggregationOperation) *aggregator { a := aggregator{ - groupBy: groupBy, digest: xxhash.New(), operation: operation, } @@ -61,7 +60,19 @@ func newAggregator(groupBy []physical.ColumnExpression, pointsSizeHint int, oper // Add adds a new sample value to the aggregation for the given timestamp and grouping label values. // It expects labelValues to be in the same order as the groupBy columns. -func (a *aggregator) Add(ts time.Time, value float64, labelValues []string) { +func (a *aggregator) Add(ts time.Time, value float64, labels []arrow.Field, labelValues []string) { + if len(labels) != len(labelValues) { + panic("len(labels) != len(labelValues)") + } + + for _, label := range labels { + if !slices.ContainsFunc(a.labels, func(l arrow.Field) bool { + return label.Equal(l) + }) { + a.labels = append(a.labels, label) + } + } + point, ok := a.points[ts] if !ok { point = make(map[uint64]*groupState) @@ -69,13 +80,15 @@ func (a *aggregator) Add(ts time.Time, value float64, labelValues []string) { } var key uint64 - if len(a.groupBy) != 0 { + if len(labelValues) != 0 { a.digest.Reset() for i, val := range labelValues { if i > 0 { _, _ = a.digest.Write([]byte{0}) // separator } + _, _ = a.digest.WriteString(labels[i].Name) + _, _ = a.digest.Write([]byte("=")) _, _ = a.digest.WriteString(val) } key = a.digest.Sum64() @@ -96,26 +109,23 @@ func (a *aggregator) Add(ts time.Time, value float64, labelValues []string) { if value < state.value { state.value = value } - case aggregationOperationCount: - state.value = state.value + 1 - + case aggregationOperationAvg: + state.value += value } + + state.count++ } else { - v := value - if a.operation == aggregationOperationCount { - v = 1 - } + count := int64(1) - if len(a.groupBy) == 0 { + if len(labels) == 0 { // special case: All values aggregated into a single group. - // This applies to queries like `sum(...)`, `sum by () (...)`, `count_over_time by () (...)`. point[key] = &groupState{ - value: v, + value: value, + count: count, } return } - // create a new slice since labelValues is reused by the calling code labelValuesCopy := make([]string, len(labelValues)) for i, v := range labelValues { // copy the value as this is backed by the arrow array data buffer. @@ -126,26 +136,27 @@ func (a *aggregator) Add(ts time.Time, value float64, labelValues []string) { // TODO: add limits on number of groups point[key] = &groupState{ + labels: labels, labelValues: labelValuesCopy, - value: v, + value: value, + count: count, } } } func (a *aggregator) BuildRecord() (arrow.RecordBatch, error) { - fields := make([]arrow.Field, 0, len(a.groupBy)+2) + fields := make([]arrow.Field, 0, len(a.labels)+2) fields = append(fields, semconv.FieldFromIdent(semconv.ColumnIdentTimestamp, false), semconv.FieldFromIdent(semconv.ColumnIdentValue, false), ) - - for _, column := range a.groupBy { - colExpr, ok := column.(*physical.ColumnExpr) - if !ok { - panic(fmt.Sprintf("invalid column expression type %T", column)) - } - ident := semconv.NewIdentifier(colExpr.Ref.Column, colExpr.Ref.Type, types.Loki.String) - fields = append(fields, semconv.FieldFromIdent(ident, true)) + for _, label := range a.labels { + fields = append(fields, arrow.Field{ + Name: label.Name, + Type: label.Type, + Nullable: true, + Metadata: label.Metadata, + }) } schema := arrow.NewSchema(fields, nil) @@ -156,16 +167,34 @@ func (a *aggregator) BuildRecord() (arrow.RecordBatch, error) { tsValue, _ := arrow.TimestampFromTime(ts, arrow.Nanosecond) for _, entry := range a.points[ts] { + var value float64 + switch a.operation { + case aggregationOperationAvg: + value = entry.value / float64(entry.count) + case aggregationOperationCount: + value = float64(entry.count) + default: + value = entry.value + } + rb.Field(0).(*array.TimestampBuilder).Append(tsValue) - rb.Field(1).(*array.Float64Builder).Append(entry.value) + rb.Field(1).(*array.Float64Builder).Append(value) + + for i, label := range a.labels { + builder := rb.Field(2 + i) // offset by 2 as the first 2 fields are timestamp and value - for col, val := range entry.labelValues { - builder := rb.Field(col + 2) // offset by 2 as the first 2 fields are timestamp and value - // TODO: differentiate between null and actual empty string - if val == "" { + j := slices.IndexFunc(entry.labels, func(l arrow.Field) bool { + return l.Name == label.Name + }) + if j == -1 { builder.(*array.StringBuilder).AppendNull() } else { - builder.(*array.StringBuilder).Append(val) + // TODO: differentiate between null and actual empty string + if entry.labelValues[j] == "" { + builder.(*array.StringBuilder).AppendNull() + } else { + builder.(*array.StringBuilder).Append(entry.labelValues[j]) + } } } } diff --git a/pkg/engine/internal/executor/aggregator_test.go b/pkg/engine/internal/executor/aggregator_test.go index bf28af0adea3c..f4f5783a97a41 100644 --- a/pkg/engine/internal/executor/aggregator_test.go +++ b/pkg/engine/internal/executor/aggregator_test.go @@ -4,28 +4,18 @@ import ( "testing" "time" + "github.com/apache/arrow-go/v18/arrow" "github.com/stretchr/testify/require" - "github.com/grafana/loki/v3/pkg/engine/internal/planner/physical" "github.com/grafana/loki/v3/pkg/engine/internal/semconv" "github.com/grafana/loki/v3/pkg/engine/internal/types" "github.com/grafana/loki/v3/pkg/util/arrowtest" ) var ( - groupBy = []physical.ColumnExpression{ - &physical.ColumnExpr{ - Ref: types.ColumnRef{ - Column: "env", - Type: types.ColumnTypeLabel, - }, - }, - &physical.ColumnExpr{ - Ref: types.ColumnRef{ - Column: "service", - Type: types.ColumnTypeLabel, - }, - }, + groupBy = []arrow.Field{ + semconv.FieldFromIdent(semconv.NewIdentifier("env", types.ColumnTypeLabel, types.Loki.String), true), + semconv.FieldFromIdent(semconv.NewIdentifier("service", types.ColumnTypeLabel, types.Loki.String), true), } ) @@ -36,25 +26,25 @@ func TestAggregator(t *testing.T) { colSvc := semconv.NewIdentifier("service", types.ColumnTypeLabel, types.Loki.String).FQN() t.Run("basic SUM aggregation with record building", func(t *testing.T) { - agg := newAggregator(groupBy, 10, aggregationOperationSum) + agg := newAggregator(10, aggregationOperationSum) ts1 := time.Date(2024, 1, 1, 10, 0, 0, 0, time.UTC) ts2 := time.Date(2024, 1, 1, 10, 1, 0, 0, time.UTC) // Add test data // ts1: prod/app1 = 10, prod/app2 = 20, dev/app1 = 30 - agg.Add(ts1, 10, []string{"prod", "app1"}) - agg.Add(ts1, 20, []string{"prod", "app2"}) - agg.Add(ts1, 30, []string{"dev", "app1"}) + agg.Add(ts1, 10, groupBy, []string{"prod", "app1"}) + agg.Add(ts1, 20, groupBy, []string{"prod", "app2"}) + agg.Add(ts1, 30, groupBy, []string{"dev", "app1"}) // ts2: prod/app1 = 15, prod/app2 = 25, dev/app2 = 35 - agg.Add(ts2, 15, []string{"prod", "app1"}) - agg.Add(ts2, 25, []string{"prod", "app2"}) - agg.Add(ts2, 35, []string{"dev", "app2"}) + agg.Add(ts2, 15, groupBy, []string{"prod", "app1"}) + agg.Add(ts2, 25, groupBy, []string{"prod", "app2"}) + agg.Add(ts2, 35, groupBy, []string{"dev", "app2"}) // Add more data to same groups to test aggregation - agg.Add(ts1, 5, []string{"prod", "app1"}) // prod/app1 at ts1 should now be 15 - agg.Add(ts2, 10, []string{"prod", "app1"}) // prod/app1 at ts2 should now be 25 + agg.Add(ts1, 5, groupBy, []string{"prod", "app1"}) // prod/app1 at ts1 should now be 15 + agg.Add(ts2, 10, groupBy, []string{"prod", "app1"}) // prod/app1 at ts2 should now be 25 record, err := agg.BuildRecord() require.NoError(t, err) @@ -75,8 +65,48 @@ func TestAggregator(t *testing.T) { require.ElementsMatch(t, expect, rows) }) + t.Run("basic AVG aggregation with record building", func(t *testing.T) { + agg := newAggregator(10, aggregationOperationAvg) + + ts1 := time.Date(2024, 1, 1, 10, 0, 0, 0, time.UTC) + ts2 := time.Date(2024, 1, 1, 10, 1, 0, 0, time.UTC) + + // Add test data + // ts1: prod/app1 = 10, prod/app2 = 20, dev/app1 = 30 + agg.Add(ts1, 10, groupBy, []string{"prod", "app1"}) + agg.Add(ts1, 20, groupBy, []string{"prod", "app2"}) + agg.Add(ts1, 30, groupBy, []string{"dev", "app1"}) + + // ts2: prod/app1 = 15, prod/app2 = 25, dev/app2 = 35 + agg.Add(ts2, 15, groupBy, []string{"prod", "app1"}) + agg.Add(ts2, 25, groupBy, []string{"prod", "app2"}) + agg.Add(ts2, 35, groupBy, []string{"dev", "app2"}) + + // Add more data to same groups to test aggregation + agg.Add(ts1, 5, groupBy, []string{"prod", "app1"}) // prod/app1 at ts1 should now be 7.5 + agg.Add(ts2, 10, groupBy, []string{"prod", "app1"}) // prod/app1 at ts2 should now be 12.5 + + record, err := agg.BuildRecord() + require.NoError(t, err) + + expect := arrowtest.Rows{ + {colTs: ts1, colVal: float64(7.5), colEnv: "prod", colSvc: "app1"}, + {colTs: ts1, colVal: float64(20), colEnv: "prod", colSvc: "app2"}, + {colTs: ts1, colVal: float64(30), colEnv: "dev", colSvc: "app1"}, + + {colTs: ts2, colVal: float64(12.5), colEnv: "prod", colSvc: "app1"}, + {colTs: ts2, colVal: float64(25), colEnv: "prod", colSvc: "app2"}, + {colTs: ts2, colVal: float64(35), colEnv: "dev", colSvc: "app2"}, + } + + rows, err := arrowtest.RecordRows(record) + require.NoError(t, err, "should be able to convert record back to rows") + require.Equal(t, len(expect), len(rows), "number of rows should match") + require.ElementsMatch(t, expect, rows) + }) + t.Run("basic COUNT aggregation with record building", func(t *testing.T) { - agg := newAggregator(groupBy, 10, aggregationOperationCount) + agg := newAggregator(10, aggregationOperationCount) ts1 := time.Date(2024, 1, 1, 10, 0, 0, 0, time.UTC) ts2 := time.Date(2024, 1, 1, 10, 1, 0, 0, time.UTC) @@ -84,24 +114,24 @@ func TestAggregator(t *testing.T) { // Add test data // ts1: add one datapoint for prod/app1, prod/app2, and dev/app1 - agg.Add(ts1, 10, []string{"prod", "app1"}) - agg.Add(ts1, 20, []string{"prod", "app2"}) - agg.Add(ts1, 30, []string{"dev", "app1"}) + agg.Add(ts1, 10, groupBy, []string{"prod", "app1"}) + agg.Add(ts1, 20, groupBy, []string{"prod", "app2"}) + agg.Add(ts1, 30, groupBy, []string{"dev", "app1"}) // ts2: add another datapoint for prod/app1, prod/app2, and dev/app2 - agg.Add(ts2, 15, []string{"prod", "app1"}) - agg.Add(ts2, 25, []string{"prod", "app2"}) - agg.Add(ts2, 35, []string{"dev", "app2"}) + agg.Add(ts2, 15, groupBy, []string{"prod", "app1"}) + agg.Add(ts2, 25, groupBy, []string{"prod", "app2"}) + agg.Add(ts2, 35, groupBy, []string{"dev", "app2"}) // ts3: add another datapoint for prod/app1, prod/app2, and dev/app2 - agg.Add(ts3, 15, []string{"prod", "app1"}) - agg.Add(ts3, 25, []string{"prod", "app2"}) - agg.Add(ts3, 35, []string{"dev", "app2"}) + agg.Add(ts3, 15, groupBy, []string{"prod", "app1"}) + agg.Add(ts3, 25, groupBy, []string{"prod", "app2"}) + agg.Add(ts3, 35, groupBy, []string{"dev", "app2"}) // Add more datapoints for prod/app1 and prod/app2 - agg.Add(ts1, 5, []string{"prod", "app1"}) // prod/app1 at ts1 should now be count 2 - agg.Add(ts2, 10, []string{"prod", "app2"}) // prod/app2 at ts2 should now be count 2 - agg.Add(ts1, 25, []string{"prod", "app1"}) // prod/app1 at ts1 should now be count 3 + agg.Add(ts1, 5, groupBy, []string{"prod", "app1"}) // prod/app1 at ts1 should now be count 2 + agg.Add(ts2, 10, groupBy, []string{"prod", "app2"}) // prod/app2 at ts2 should now be count 2 + agg.Add(ts1, 25, groupBy, []string{"prod", "app1"}) // prod/app1 at ts1 should now be count 3 record, err := agg.BuildRecord() require.NoError(t, err) @@ -127,26 +157,26 @@ func TestAggregator(t *testing.T) { }) t.Run("basic MAX aggregation with record building", func(t *testing.T) { - agg := newAggregator(groupBy, 10, aggregationOperationMax) + agg := newAggregator(10, aggregationOperationMax) ts1 := time.Date(2024, 1, 1, 10, 0, 0, 0, time.UTC) ts2 := time.Date(2024, 1, 1, 10, 1, 0, 0, time.UTC) // Add test data // ts1: add one datapoint for prod/app1, prod/app2, and dev/app1 - agg.Add(ts1, 10, []string{"prod", "app1"}) - agg.Add(ts1, 20, []string{"prod", "app2"}) - agg.Add(ts1, 30, []string{"dev", "app1"}) + agg.Add(ts1, 10, groupBy, []string{"prod", "app1"}) + agg.Add(ts1, 20, groupBy, []string{"prod", "app2"}) + agg.Add(ts1, 30, groupBy, []string{"dev", "app1"}) // ts2: add another datapoint for prod/app1, prod/app2, and dev/app2 - agg.Add(ts2, 15, []string{"prod", "app1"}) - agg.Add(ts2, 25, []string{"prod", "app2"}) - agg.Add(ts2, 35, []string{"dev", "app2"}) + agg.Add(ts2, 15, groupBy, []string{"prod", "app1"}) + agg.Add(ts2, 25, groupBy, []string{"prod", "app2"}) + agg.Add(ts2, 35, groupBy, []string{"dev", "app2"}) // Add more datapoints for prod/app1 and prod/app2 - agg.Add(ts1, 5, []string{"prod", "app1"}) // prod/app1 at ts1 should still be 10 - agg.Add(ts2, 50, []string{"prod", "app2"}) // prod/app2 at ts2 should now be 50 - agg.Add(ts1, 15, []string{"prod", "app1"}) // prod/app1 at ts1 should now be 15 + agg.Add(ts1, 5, groupBy, []string{"prod", "app1"}) // prod/app1 at ts1 should still be 10 + agg.Add(ts2, 50, groupBy, []string{"prod", "app2"}) // prod/app2 at ts2 should now be 50 + agg.Add(ts1, 15, groupBy, []string{"prod", "app1"}) // prod/app1 at ts1 should now be 15 record, err := agg.BuildRecord() require.NoError(t, err) @@ -168,26 +198,26 @@ func TestAggregator(t *testing.T) { }) t.Run("basic MIN aggregation with record building", func(t *testing.T) { - agg := newAggregator(groupBy, 10, aggregationOperationMin) + agg := newAggregator(10, aggregationOperationMin) ts1 := time.Date(2024, 1, 1, 10, 0, 0, 0, time.UTC) ts2 := time.Date(2024, 1, 1, 10, 1, 0, 0, time.UTC) // Add test data // ts1: add one datapoint for prod/app1, prod/app2, and dev/app1 - agg.Add(ts1, 10, []string{"prod", "app1"}) - agg.Add(ts1, 20, []string{"prod", "app2"}) - agg.Add(ts1, 30, []string{"dev", "app1"}) + agg.Add(ts1, 10, groupBy, []string{"prod", "app1"}) + agg.Add(ts1, 20, groupBy, []string{"prod", "app2"}) + agg.Add(ts1, 30, groupBy, []string{"dev", "app1"}) // ts2: add another datapoint for prod/app1, prod/app2, and dev/app2 - agg.Add(ts2, 15, []string{"prod", "app1"}) - agg.Add(ts2, 25, []string{"prod", "app2"}) - agg.Add(ts2, 35, []string{"dev", "app2"}) + agg.Add(ts2, 15, groupBy, []string{"prod", "app1"}) + agg.Add(ts2, 25, groupBy, []string{"prod", "app2"}) + agg.Add(ts2, 35, groupBy, []string{"dev", "app2"}) // Add more datapoints for prod/app1 and prod/app2 - agg.Add(ts1, 5, []string{"prod", "app1"}) // prod/app1 at ts1 should now be 5 - agg.Add(ts2, 40, []string{"prod", "app2"}) // prod/app2 at ts2 should still be 25 - agg.Add(ts1, 25, []string{"prod", "app1"}) // prod/app1 at ts1 should still be 5 + agg.Add(ts1, 5, groupBy, []string{"prod", "app1"}) // prod/app1 at ts1 should now be 5 + agg.Add(ts2, 40, groupBy, []string{"prod", "app2"}) // prod/app2 at ts2 should still be 25 + agg.Add(ts1, 25, groupBy, []string{"prod", "app1"}) // prod/app1 at ts1 should still be 5 record, err := agg.BuildRecord() require.NoError(t, err) @@ -210,26 +240,26 @@ func TestAggregator(t *testing.T) { t.Run("SUM aggregation with empty groupBy", func(t *testing.T) { // Empty groupBy represents sum by () or sum(...) - all values aggregated into single group - groupBy := []physical.ColumnExpression{} + groupBy := []arrow.Field{} - agg := newAggregator(groupBy, 1, aggregationOperationSum) + agg := newAggregator(1, aggregationOperationSum) ts1 := time.Date(2024, 1, 1, 10, 0, 0, 0, time.UTC) ts2 := time.Date(2024, 1, 1, 10, 1, 0, 0, time.UTC) // Add test data // ts1: prod/app1 = 10, prod/app2 = 20, dev/app1 = 30 - agg.Add(ts1, 10, []string{"prod", "app1"}) - agg.Add(ts1, 20, []string{"prod", "app2"}) - agg.Add(ts1, 30, []string{"dev", "app1"}) + agg.Add(ts1, 10, groupBy, []string{}) // "prod", "app1" + agg.Add(ts1, 20, groupBy, []string{}) // "prod", "app2" + agg.Add(ts1, 30, groupBy, []string{}) // "dev", "app1" // ts2: prod/app1 = 15, prod/app2 = 25, dev/app2 = 35 - agg.Add(ts2, 15, []string{"prod", "app1"}) - agg.Add(ts2, 25, []string{"prod", "app2"}) - agg.Add(ts2, 35, []string{"dev", "app2"}) + agg.Add(ts2, 15, groupBy, []string{}) // "prod", "app1" + agg.Add(ts2, 25, groupBy, []string{}) // "prod", "app2" + agg.Add(ts2, 35, groupBy, []string{}) // "dev", "app2" - agg.Add(ts1, 5, []string{"prod", "app1"}) - agg.Add(ts2, 10, []string{"prod", "app1"}) + agg.Add(ts1, 5, groupBy, []string{}) // "prod", "app1" + agg.Add(ts2, 10, groupBy, []string{}) // "prod", "app1" record, err := agg.BuildRecord() require.NoError(t, err) diff --git a/pkg/engine/internal/executor/cast.go b/pkg/engine/internal/executor/cast.go index 1b1021165b262..b0baf7d906967 100644 --- a/pkg/engine/internal/executor/cast.go +++ b/pkg/engine/internal/executor/cast.go @@ -154,10 +154,8 @@ func (et *errorTracker) recordError(rowIndex int, err error) { et.errorBuilder = array.NewStringBuilder(memory.DefaultAllocator) et.detailsBuilder = array.NewStringBuilder(memory.DefaultAllocator) // Backfill nulls for previous rows - for range rowIndex { - et.errorBuilder.AppendNull() - et.detailsBuilder.AppendNull() - } + et.errorBuilder.AppendNulls(rowIndex) + et.detailsBuilder.AppendNulls(rowIndex) et.hasErrors = true } et.errorBuilder.Append(types.SampleExtractionErrorType) diff --git a/pkg/engine/internal/executor/executor.go b/pkg/engine/internal/executor/executor.go index fd4024695f18a..539661777d746 100644 --- a/pkg/engine/internal/executor/executor.go +++ b/pkg/engine/internal/executor/executor.go @@ -324,7 +324,8 @@ func (c *Context) executeProjection(ctx context.Context, proj *physical.Projecti func (c *Context) executeRangeAggregation(ctx context.Context, plan *physical.RangeAggregation, inputs []Pipeline, region *xcap.Region) Pipeline { ctx, span := tracer.Start(ctx, "Context.executeRangeAggregation", trace.WithAttributes( - attribute.Int("num_partition_by", len(plan.PartitionBy)), + attribute.Int("num_grouping", len(plan.Grouping.Columns)), + attribute.String("grouping_mode", plan.Grouping.Mode.String()), attribute.Int64("start_ts", plan.Start.UnixNano()), attribute.Int64("end_ts", plan.End.UnixNano()), attribute.Int64("range_interval", int64(plan.Range)), @@ -338,7 +339,7 @@ func (c *Context) executeRangeAggregation(ctx context.Context, plan *physical.Ra } pipeline, err := newRangeAggregationPipeline(inputs, c.evaluator, rangeAggregationOptions{ - partitionBy: plan.PartitionBy, + grouping: plan.Grouping, startTs: plan.Start, endTs: plan.End, rangeInterval: plan.Range, @@ -354,7 +355,8 @@ func (c *Context) executeRangeAggregation(ctx context.Context, plan *physical.Ra func (c *Context) executeVectorAggregation(ctx context.Context, plan *physical.VectorAggregation, inputs []Pipeline, region *xcap.Region) Pipeline { ctx, span := tracer.Start(ctx, "Context.executeVectorAggregation", trace.WithAttributes( - attribute.Int("num_group_by", len(plan.GroupBy)), + attribute.Int("num_grouping", len(plan.Grouping.Columns)), + attribute.String("grouping_mode", plan.Grouping.Mode.String()), attribute.Int("num_inputs", len(inputs)), )) defer span.End() @@ -363,7 +365,7 @@ func (c *Context) executeVectorAggregation(ctx context.Context, plan *physical.V return emptyPipeline() } - pipeline, err := newVectorAggregationPipeline(inputs, plan.GroupBy, c.evaluator, plan.Operation, region) + pipeline, err := newVectorAggregationPipeline(inputs, plan.Grouping, c.evaluator, plan.Operation, region) if err != nil { return errorPipeline(ctx, err) } @@ -490,13 +492,15 @@ func startRegionForNode(ctx context.Context, n physical.Node) (context.Context, attribute.Int64("end_ts", n.End.UnixNano()), attribute.Int64("range_interval", int64(n.Range)), attribute.Int64("step", int64(n.Step)), - attribute.Int("num_partition_by", len(n.PartitionBy)), + attribute.Int("num_grouping", len(n.Grouping.Columns)), + attribute.String("grouping_mode", n.Grouping.Mode.String()), ) case *physical.VectorAggregation: attributes = append(attributes, attribute.String("operation", string(rune(n.Operation))), - attribute.Int("num_group_by", len(n.GroupBy)), + attribute.Int("num_grouping", len(n.Grouping.Columns)), + attribute.String("grouping_mode", n.Grouping.Mode.String()), ) case *physical.ColumnCompat: diff --git a/pkg/engine/internal/executor/range_aggregation.go b/pkg/engine/internal/executor/range_aggregation.go index 3bd43d94baae7..99aaca0931417 100644 --- a/pkg/engine/internal/executor/range_aggregation.go +++ b/pkg/engine/internal/executor/range_aggregation.go @@ -11,13 +11,15 @@ import ( "github.com/apache/arrow-go/v18/arrow" "github.com/apache/arrow-go/v18/arrow/array" + "github.com/grafana/loki/v3/pkg/engine/internal/semconv" + "github.com/grafana/loki/v3/pkg/engine/internal/planner/physical" "github.com/grafana/loki/v3/pkg/engine/internal/types" "github.com/grafana/loki/v3/pkg/xcap" ) type rangeAggregationOptions struct { - partitionBy []physical.ColumnExpression + grouping physical.Grouping // start and end timestamps are equal for instant queries. startTs time.Time // start timestamp of the query @@ -32,8 +34,6 @@ var ( rangeAggregationOperations = map[types.RangeAggregationType]aggregationOperation{ types.RangeAggregationTypeSum: aggregationOperationSum, types.RangeAggregationTypeCount: aggregationOperationCount, - types.RangeAggregationTypeMax: aggregationOperationMax, - types.RangeAggregationTypeMin: aggregationOperationMin, } ) @@ -56,8 +56,8 @@ type timestampMatchingWindowsFunc func(time.Time) []window // rangeAggregationPipeline is a pipeline that performs aggregations over a time window. // // 1. It reads from the input pipelines -// 2. Partitions the data by the specified columns -// 3. Applies the aggregation function on each partition +// 2. Groups the data by the specified columns +// 3. Applies the aggregation function on each group // // Current version only supports counting for instant queries. type rangeAggregationPipeline struct { @@ -104,7 +104,7 @@ func (r *rangeAggregationPipeline) init() { panic(fmt.Sprintf("unknown range aggregation operation: %v", r.opts.operation)) } - r.aggregator = newAggregator(r.opts.partitionBy, len(windows), op) + r.aggregator = newAggregator(len(windows), op) } // Read reads the next value into its state. @@ -119,7 +119,6 @@ func (r *rangeAggregationPipeline) Read(ctx context.Context) (arrow.RecordBatch, } // TODOs: -// - Support implicit partitioning by all labels when partitionBy is empty // - Use columnar access pattern. Current approach is row-based which does not benefit from the storage format. // - Add toggle to return partial results on Read() call instead of returning only after exhausting all inputs. func (r *rangeAggregationPipeline) read(ctx context.Context) (arrow.RecordBatch, error) { @@ -137,9 +136,6 @@ func (r *rangeAggregationPipeline) read(ctx context.Context) (arrow.RecordBatch, Type: types.ColumnTypeGenerated, }, } // value column expression - - // reused on each row read - labelValues = make([]string, len(r.opts.partitionBy)) ) r.aggregator.Reset() // reset before reading new inputs @@ -158,20 +154,74 @@ func (r *rangeAggregationPipeline) read(ctx context.Context) (arrow.RecordBatch, inputsExhausted = false - // extract all the columns that are used for partitioning - arrays := make([]*array.String, 0, len(r.opts.partitionBy)) - for _, columnExpr := range r.opts.partitionBy { - vec, err := r.evaluator.eval(columnExpr, record) - if err != nil { - return nil, err + // extract all the columns that are used for grouping + var arrays []*array.String + var fields []arrow.Field + switch r.opts.grouping.Mode { + case types.GroupingModeByLabelSet: + // Gouping by a label set. Take only labels from that set. + arrays = make([]*array.String, 0, len(r.opts.grouping.Columns)) + for _, columnExpr := range r.opts.grouping.Columns { + vec, err := r.evaluator.eval(columnExpr, record) + if err != nil { + return nil, err + } + + if vec.DataType().ID() != types.Arrow.String.ID() { + return nil, fmt.Errorf("unsupported datatype for grouping %s", vec.DataType()) + } + + arr := vec.(*array.String) + arrays = append(arrays, arr) + + colExpr, ok := columnExpr.(*physical.ColumnExpr) + if !ok { + return nil, fmt.Errorf("invalid column expression type %T", columnExpr) + } + ident := semconv.NewIdentifier(colExpr.Ref.Column, colExpr.Ref.Type, types.Loki.String) + fields = append(fields, semconv.FieldFromIdent(ident, true)) } - - if vec.DataType().ID() != types.Arrow.String.ID() { - return nil, fmt.Errorf("unsupported datatype for partitioning %s", vec.DataType()) + case types.GroupingModeByEmptySet: + // Gouping by an empty set. Group all into one. + arrays = make([]*array.String, 0) + fields = make([]arrow.Field, 0) + case types.GroupingModeWithoutLabelSet, types.GroupingModeWithoutEmptySet: + // Grouping without a lable set. Exclude lables from that set. + schema := record.Schema() + for i, field := range schema.Fields() { + ident, err := semconv.ParseFQN(field.Name) + if err != nil { + return nil, err + } + + if ident.ColumnType() == types.ColumnTypeLabel || + ident.ColumnType() == types.ColumnTypeMetadata || + ident.ColumnType() == types.ColumnTypeParsed { + found := false + for _, g := range r.opts.grouping.Columns { + colExpr, ok := g.(*physical.ColumnExpr) + if !ok { + return nil, fmt.Errorf("unknown column expression %v", g) + } + + // Match ambiguous columns only by name + if colExpr.Ref.Type == types.ColumnTypeAmbiguous && colExpr.Ref.Column == ident.ShortName() { + found = true + break + } + + // Match all other columns by name and type + if colExpr.Ref.Column == ident.ShortName() && colExpr.Ref.Type == ident.ColumnType() { + found = true + break + } + } + if !found { + arrays = append(arrays, record.Column(i).(*array.String)) + fields = append(fields, field) + } + } } - - arr := vec.(*array.String) - arrays = append(arrays, arr) } // extract timestamp column to check if the entry is in range @@ -192,24 +242,32 @@ func (r *rangeAggregationPipeline) read(ctx context.Context) (arrow.RecordBatch, } for row := range int(record.NumRows()) { + var value float64 + if r.opts.operation != types.RangeAggregationTypeCount { + if valArr.IsNull(row) { + continue + } + + value = valArr.Value(row) + } + windows := r.windowsForTimestamp(tsCol.Value(row).ToTime(arrow.Nanosecond)) if len(windows) == 0 { continue // out of range, skip this row } - // reset label values and hash for each row - clear(labelValues) - for col, arr := range arrays { - labelValues[col] = arr.Value(row) - } - - var value float64 - if r.opts.operation != types.RangeAggregationTypeCount { - value = valArr.Value(row) + labelValues := make([]string, 0, len(arrays)) + labels := make([]arrow.Field, 0, len(arrays)) + for i, arr := range arrays { + val := arr.Value(row) + if val != "" { + labelValues = append(labelValues, val) + labels = append(labels, fields[i]) + } } for _, w := range windows { - r.aggregator.Add(w.end, value, labelValues) + r.aggregator.Add(w.end, value, labels, labelValues) } } } diff --git a/pkg/engine/internal/executor/range_aggregation_test.go b/pkg/engine/internal/executor/range_aggregation_test.go index b8cf424d5a36c..61fb8e9c07fa4 100644 --- a/pkg/engine/internal/executor/range_aggregation_test.go +++ b/pkg/engine/internal/executor/range_aggregation_test.go @@ -56,19 +56,22 @@ func TestRangeAggregationPipeline_instant(t *testing.T) { } opts := rangeAggregationOptions{ - partitionBy: []physical.ColumnExpression{ - &physical.ColumnExpr{ - Ref: types.ColumnRef{ - Column: "env", - Type: types.ColumnTypeAmbiguous, + grouping: physical.Grouping{ + Columns: []physical.ColumnExpression{ + &physical.ColumnExpr{ + Ref: types.ColumnRef{ + Column: "env", + Type: types.ColumnTypeAmbiguous, + }, }, - }, - &physical.ColumnExpr{ - Ref: types.ColumnRef{ - Column: "service", - Type: types.ColumnTypeAmbiguous, + &physical.ColumnExpr{ + Ref: types.ColumnRef{ + Column: "service", + Type: types.ColumnTypeAmbiguous, + }, }, }, + Mode: types.GroupingModeByLabelSet, }, startTs: time.Unix(20, 0).UTC(), endTs: time.Unix(20, 0).UTC(), @@ -138,7 +141,7 @@ func TestRangeAggregationPipeline(t *testing.T) { }} ) - partitionBy := []physical.ColumnExpression{ + groupBy := []physical.ColumnExpression{ &physical.ColumnExpr{ Ref: types.ColumnRef{ Column: "env", @@ -155,7 +158,10 @@ func TestRangeAggregationPipeline(t *testing.T) { t.Run("aligned windows", func(t *testing.T) { opts := rangeAggregationOptions{ - partitionBy: partitionBy, + grouping: physical.Grouping{ + Columns: groupBy, + Mode: types.GroupingModeByLabelSet, + }, startTs: time.Unix(10, 0), endTs: time.Unix(40, 0), rangeInterval: 10 * time.Second, @@ -202,7 +208,10 @@ func TestRangeAggregationPipeline(t *testing.T) { t.Run("overlapping windows", func(t *testing.T) { opts := rangeAggregationOptions{ - partitionBy: partitionBy, + grouping: physical.Grouping{ + Columns: groupBy, + Mode: types.GroupingModeByLabelSet, + }, startTs: time.Unix(10, 0), endTs: time.Unix(40, 0), rangeInterval: 10 * time.Second, @@ -263,7 +272,10 @@ func TestRangeAggregationPipeline(t *testing.T) { t.Run("non-overlapping windows", func(t *testing.T) { opts := rangeAggregationOptions{ - partitionBy: partitionBy, + grouping: physical.Grouping{ + Columns: groupBy, + Mode: types.GroupingModeByLabelSet, + }, startTs: time.Unix(10, 0), endTs: time.Unix(40, 0), rangeInterval: 5 * time.Second, diff --git a/pkg/engine/internal/executor/vector_aggregate.go b/pkg/engine/internal/executor/vector_aggregate.go index a97d9aa4ad24a..9ffceed2dee0f 100644 --- a/pkg/engine/internal/executor/vector_aggregate.go +++ b/pkg/engine/internal/executor/vector_aggregate.go @@ -8,6 +8,8 @@ import ( "github.com/apache/arrow-go/v18/arrow" "github.com/apache/arrow-go/v18/arrow/array" + "github.com/grafana/loki/v3/pkg/engine/internal/semconv" + "github.com/grafana/loki/v3/pkg/engine/internal/planner/physical" "github.com/grafana/loki/v3/pkg/engine/internal/types" "github.com/grafana/loki/v3/pkg/xcap" @@ -23,7 +25,7 @@ type vectorAggregationPipeline struct { aggregator *aggregator evaluator expressionEvaluator - groupBy []physical.ColumnExpression + grouping physical.Grouping region *xcap.Region tsEval evalFunc // used to evaluate the timestamp column @@ -34,12 +36,10 @@ var ( vectorAggregationOperations = map[types.VectorAggregationType]aggregationOperation{ types.VectorAggregationTypeSum: aggregationOperationSum, types.VectorAggregationTypeCount: aggregationOperationCount, - types.VectorAggregationTypeMax: aggregationOperationMax, - types.VectorAggregationTypeMin: aggregationOperationMin, } ) -func newVectorAggregationPipeline(inputs []Pipeline, groupBy []physical.ColumnExpression, evaluator expressionEvaluator, operation types.VectorAggregationType, region *xcap.Region) (*vectorAggregationPipeline, error) { +func newVectorAggregationPipeline(inputs []Pipeline, grouping physical.Grouping, evaluator expressionEvaluator, operation types.VectorAggregationType, region *xcap.Region) (*vectorAggregationPipeline, error) { if len(inputs) == 0 { return nil, fmt.Errorf("vector aggregation expects at least one input") } @@ -52,8 +52,8 @@ func newVectorAggregationPipeline(inputs []Pipeline, groupBy []physical.ColumnEx return &vectorAggregationPipeline{ inputs: inputs, evaluator: evaluator, - groupBy: groupBy, - aggregator: newAggregator(groupBy, 0, op), + grouping: grouping, + aggregator: newAggregator(0, op), region: region, tsEval: evaluator.newFunc(&physical.ColumnExpr{ Ref: types.ColumnRef{ @@ -79,10 +79,6 @@ func (v *vectorAggregationPipeline) Read(ctx context.Context) (arrow.RecordBatch } func (v *vectorAggregationPipeline) read(ctx context.Context) (arrow.RecordBatch, error) { - var ( - labelValues = make([]string, len(v.groupBy)) - ) - v.aggregator.Reset() // reset before reading new inputs inputsExhausted := false for !inputsExhausted { @@ -114,30 +110,91 @@ func (v *vectorAggregationPipeline) read(ctx context.Context) (arrow.RecordBatch valueArr := valueVec.(*array.Float64) // extract all the columns that are used for grouping - arrays := make([]*array.String, 0, len(v.groupBy)) - - for _, columnExpr := range v.groupBy { - vec, err := v.evaluator.eval(columnExpr, record) - if err != nil { - return nil, err + var arrays []*array.String + var fields []arrow.Field + switch v.grouping.Mode { + case types.GroupingModeByLabelSet: + // Gouping by a label set. Take only labels from that set. + arrays = make([]*array.String, 0, len(v.grouping.Columns)) + for _, columnExpr := range v.grouping.Columns { + vec, err := v.evaluator.eval(columnExpr, record) + if err != nil { + return nil, err + } + + if vec.DataType().ID() != types.Arrow.String.ID() { + return nil, fmt.Errorf("unsupported datatype for grouping %s", vec.DataType()) + } + + arr := vec.(*array.String) + arrays = append(arrays, arr) + + colExpr, ok := columnExpr.(*physical.ColumnExpr) + if !ok { + return nil, fmt.Errorf("invalid column expression type %T", columnExpr) + } + ident := semconv.NewIdentifier(colExpr.Ref.Column, colExpr.Ref.Type, types.Loki.String) + fields = append(fields, semconv.FieldFromIdent(ident, true)) } - - if vec.DataType().ID() != types.Arrow.String.ID() { - return nil, fmt.Errorf("unsupported datatype for grouping %s", vec.DataType()) + case types.GroupingModeByEmptySet: + // Gouping by an empty set. Group all into one. + arrays = make([]*array.String, 0) + fields = make([]arrow.Field, 0) + case types.GroupingModeWithoutLabelSet, types.GroupingModeWithoutEmptySet: + // Grouping without a lable set. Exclude lables from that set. + schema := record.Schema() + for i, field := range schema.Fields() { + ident, err := semconv.ParseFQN(field.Name) + if err != nil { + return nil, err + } + + if ident.ColumnType() == types.ColumnTypeLabel || + ident.ColumnType() == types.ColumnTypeMetadata || + ident.ColumnType() == types.ColumnTypeParsed { + found := false + for _, g := range v.grouping.Columns { + colExpr, ok := g.(*physical.ColumnExpr) + if !ok { + return nil, fmt.Errorf("unknown column expression %v", g) + } + + // Match ambiguous columns only by name + if colExpr.Ref.Type == types.ColumnTypeAmbiguous && colExpr.Ref.Column == ident.ShortName() { + found = true + break + } + + // Match all other columns by name and type + if colExpr.Ref.Column == ident.ShortName() && colExpr.Ref.Type == ident.ColumnType() { + found = true + break + } + } + if !found { + arrays = append(arrays, record.Column(i).(*array.String)) + fields = append(fields, field) + } + } } - - arr := vec.(*array.String) - arrays = append(arrays, arr) } for row := range int(record.NumRows()) { - // reset for each row - clear(labelValues) - for col, arr := range arrays { - labelValues[col] = arr.Value(row) + if valueArr.IsNull(row) { + continue + } + + labelValues := make([]string, 0, len(arrays)) + labels := make([]arrow.Field, 0, len(arrays)) + for i, arr := range arrays { + val := arr.Value(row) + if val != "" { + labelValues = append(labelValues, val) + labels = append(labels, fields[i]) + } } - v.aggregator.Add(tsCol.Value(row).ToTime(arrow.Nanosecond), valueArr.Value(row), labelValues) + v.aggregator.Add(tsCol.Value(row).ToTime(arrow.Nanosecond), valueArr.Value(row), labels, labelValues) } } } diff --git a/pkg/engine/internal/executor/vector_aggregate_test.go b/pkg/engine/internal/executor/vector_aggregate_test.go index 25db233dfd0bb..22c78e75f27d1 100644 --- a/pkg/engine/internal/executor/vector_aggregate_test.go +++ b/pkg/engine/internal/executor/vector_aggregate_test.go @@ -67,22 +67,25 @@ func TestVectorAggregationPipeline(t *testing.T) { input2 := NewBufferedPipeline(input2Record) // Create group by expressions - groupBy := []physical.ColumnExpression{ - &physical.ColumnExpr{ - Ref: types.ColumnRef{ - Column: "env", - Type: types.ColumnTypeAmbiguous, + grouping := physical.Grouping{ + Columns: []physical.ColumnExpression{ + &physical.ColumnExpr{ + Ref: types.ColumnRef{ + Column: "env", + Type: types.ColumnTypeAmbiguous, + }, }, - }, - &physical.ColumnExpr{ - Ref: types.ColumnRef{ - Column: "service", - Type: types.ColumnTypeAmbiguous, + &physical.ColumnExpr{ + Ref: types.ColumnRef{ + Column: "service", + Type: types.ColumnTypeAmbiguous, + }, }, }, + Mode: types.GroupingModeByLabelSet, } - pipeline, err := newVectorAggregationPipeline([]Pipeline{input1, input2}, groupBy, newExpressionEvaluator(), types.VectorAggregationTypeSum, nil) + pipeline, err := newVectorAggregationPipeline([]Pipeline{input1, input2}, grouping, newExpressionEvaluator(), types.VectorAggregationTypeSum, nil) require.NoError(t, err) defer pipeline.Close() diff --git a/pkg/engine/internal/planner/logical/builder.go b/pkg/engine/internal/planner/logical/builder.go index 5ce753dca493f..4214372e1d644 100644 --- a/pkg/engine/internal/planner/logical/builder.go +++ b/pkg/engine/internal/planner/logical/builder.go @@ -122,7 +122,7 @@ func (b *Builder) BinOpLeft(op types.BinaryOp, left Value) *Builder { // RangeAggregation applies a [RangeAggregation] operation to the Builder. func (b *Builder) RangeAggregation( - partitionBy []ColumnRef, + grouping Grouping, operation types.RangeAggregationType, startTS, endTS time.Time, step time.Duration, @@ -133,7 +133,7 @@ func (b *Builder) RangeAggregation( Table: b.val, Operation: operation, - PartitionBy: partitionBy, + Grouping: grouping, Start: startTS, End: endTS, Step: step, @@ -144,13 +144,13 @@ func (b *Builder) RangeAggregation( // VectorAggregation applies a [VectorAggregation] operation to the Builder. func (b *Builder) VectorAggregation( - groupBy []ColumnRef, + grouping Grouping, operation types.VectorAggregationType, ) *Builder { return &Builder{ val: &VectorAggregation{ Table: b.val, - GroupBy: groupBy, + Grouping: grouping, Operation: operation, }, } diff --git a/pkg/engine/internal/planner/logical/format_tree.go b/pkg/engine/internal/planner/logical/format_tree.go index 3b33e38b3848c..436f0b26fd236 100644 --- a/pkg/engine/internal/planner/logical/format_tree.go +++ b/pkg/engine/internal/planner/logical/format_tree.go @@ -4,6 +4,7 @@ import ( "fmt" "io" + "github.com/grafana/loki/v3/pkg/engine/internal/types" "github.com/grafana/loki/v3/pkg/engine/internal/util" "github.com/grafana/loki/v3/pkg/engine/internal/util/tree" ) @@ -170,17 +171,21 @@ func (t *treeFormatter) convertRangeAggregation(r *RangeAggregation) *tree.Node tree.NewProperty("range", false, r.RangeInterval), } - if len(r.PartitionBy) > 0 { - partitionBy := make([]any, len(r.PartitionBy)) - for i := range r.PartitionBy { - partitionBy[i] = r.PartitionBy[i].Name() + grouping := make([]any, len(r.Grouping.Columns)) + if len(r.Grouping.Columns) > 0 { + for i := range r.Grouping.Columns { + grouping[i] = r.Grouping.Columns[i].Name() } - - properties = append(properties, tree.NewProperty("partition_by", true, partitionBy...)) + } + switch r.Grouping.Mode { + case types.GroupingModeByLabelSet, types.GroupingModeByEmptySet: + properties = append(properties, tree.NewProperty("group_by", true, grouping...)) + case types.GroupingModeWithoutLabelSet, types.GroupingModeWithoutEmptySet: + properties = append(properties, tree.NewProperty("group_without", true, grouping...)) } node := tree.NewNode("RangeAggregation", r.Name(), properties...) - for _, columnRef := range r.PartitionBy { + for _, columnRef := range r.Grouping.Columns { node.Comments = append(node.Comments, t.convert(&columnRef)) } node.Children = append(node.Children, t.convert(r.Table)) @@ -194,17 +199,21 @@ func (t *treeFormatter) convertVectorAggregation(v *VectorAggregation) *tree.Nod tree.NewProperty("operation", false, v.Operation), } - if len(v.GroupBy) > 0 { - groupBy := make([]any, len(v.GroupBy)) - for i := range v.GroupBy { - groupBy[i] = v.GroupBy[i].Name() + grouping := make([]any, len(v.Grouping.Columns)) + if len(v.Grouping.Columns) > 0 { + for i := range v.Grouping.Columns { + grouping[i] = v.Grouping.Columns[i].Name() } - - properties = append(properties, tree.NewProperty("group_by", true, groupBy...)) + } + switch v.Grouping.Mode { + case types.GroupingModeByLabelSet, types.GroupingModeByEmptySet: + properties = append(properties, tree.NewProperty("group_by", true, grouping...)) + case types.GroupingModeWithoutLabelSet, types.GroupingModeWithoutEmptySet: + properties = append(properties, tree.NewProperty("group_without", true, grouping...)) } node := tree.NewNode("VectorAggregation", v.Name(), properties...) - for _, columnRef := range v.GroupBy { + for _, columnRef := range v.Grouping.Columns { node.Comments = append(node.Comments, t.convert(&columnRef)) } node.Children = append(node.Children, t.convert(v.Table)) diff --git a/pkg/engine/internal/planner/logical/format_tree_test.go b/pkg/engine/internal/planner/logical/format_tree_test.go index dd95770df092c..13d312f705a85 100644 --- a/pkg/engine/internal/planner/logical/format_tree_test.go +++ b/pkg/engine/internal/planner/logical/format_tree_test.go @@ -115,7 +115,10 @@ func TestFormatRangeAggregationQuery(t *testing.T) { Op: types.BinaryOpGt, }, ).RangeAggregation( - []ColumnRef{*NewColumnRef("label1", types.ColumnTypeAmbiguous), *NewColumnRef("label2", types.ColumnTypeAmbiguous)}, + Grouping{ + Columns: []ColumnRef{*NewColumnRef("label1", types.ColumnTypeAmbiguous), *NewColumnRef("label2", types.ColumnTypeAmbiguous)}, + Mode: types.GroupingModeByLabelSet, + }, types.RangeAggregationTypeCount, time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC), // Start Time time.Date(1970, 1, 1, 1, 0, 0, 0, time.UTC), // End Time @@ -134,7 +137,7 @@ func TestFormatRangeAggregationQuery(t *testing.T) { t.Logf("Actual output:\n%s", actual) expected := ` -RangeAggregation <%5> table=%4 operation=count start_ts=1970-01-01T00:00:00Z end_ts=1970-01-01T01:00:00Z step=1m0s range=5m0s partition_by=(ambiguous.label1, ambiguous.label2) +RangeAggregation <%5> table=%4 operation=count start_ts=1970-01-01T00:00:00Z end_ts=1970-01-01T01:00:00Z step=1m0s range=5m0s group_by=(ambiguous.label1, ambiguous.label2) │ ├── ColumnRef column=label1 type=ambiguous │ └── ColumnRef column=label2 type=ambiguous └── SELECT <%4> table=%2 predicate=%3 @@ -163,9 +166,12 @@ func TestFormatVectorAggregationQuery(t *testing.T) { }, }, ).VectorAggregation( - []ColumnRef{ - *NewColumnRef("app", types.ColumnTypeLabel), - *NewColumnRef("env", types.ColumnTypeLabel), + Grouping{ + Columns: []ColumnRef{ + *NewColumnRef("app", types.ColumnTypeLabel), + *NewColumnRef("env", types.ColumnTypeLabel), + }, + Mode: types.GroupingModeByLabelSet, }, types.VectorAggregationTypeSum, ) diff --git a/pkg/engine/internal/planner/logical/grouping.go b/pkg/engine/internal/planner/logical/grouping.go new file mode 100644 index 0000000000000..c31a7efe8311d --- /dev/null +++ b/pkg/engine/internal/planner/logical/grouping.go @@ -0,0 +1,13 @@ +package logical + +import "github.com/grafana/loki/v3/pkg/engine/internal/types" + +// Grouping represents the grouping by/without label(s) for vector aggregators and range vector aggregators. +type Grouping struct { + Columns []ColumnRef // The columns for grouping + Mode types.GroupingMode // The grouping mode +} + +var ( + NoGrouping = Grouping{Mode: types.GroupingModeWithoutEmptySet} +) diff --git a/pkg/engine/internal/planner/logical/node_range_aggregate.go b/pkg/engine/internal/planner/logical/node_range_aggregate.go index fa3db993ef6bc..1993795fed548 100644 --- a/pkg/engine/internal/planner/logical/node_range_aggregate.go +++ b/pkg/engine/internal/planner/logical/node_range_aggregate.go @@ -12,12 +12,11 @@ import ( // It is similar to window functions in SQL with a few important distinctions: // 1. It evaluates the aggregation at step intervals unlike traditional window functions which are evaluated for each row. // 2. It uses a time window defined by query [$range]. -// 3. It partitions by query-time streams if no partition by is specified. type RangeAggregation struct { id string - Table Value // The table relation to aggregate. - PartitionBy []ColumnRef // The columns to partition by. + Table Value // The table relation to aggregate. + Grouping Grouping // The grouping Operation types.RangeAggregationType // The type of aggregation operation to perform. Start time.Time @@ -43,16 +42,20 @@ func (r *RangeAggregation) Name() string { func (r *RangeAggregation) String() string { props := fmt.Sprintf("operation=%s, start_ts=%s, end_ts=%s, step=%s, range=%s", r.Operation, util.FormatTimeRFC3339Nano(r.Start), util.FormatTimeRFC3339Nano(r.End), r.Step, r.RangeInterval) - if len(r.PartitionBy) > 0 { - partitionBy := "" - for i, columnRef := range r.PartitionBy { + grouping := "" + if len(r.Grouping.Columns) > 0 { + for i, columnRef := range r.Grouping.Columns { if i > 0 { - partitionBy += ", " + grouping += ", " } - partitionBy += columnRef.String() + grouping += columnRef.String() } - - return fmt.Sprintf("RANGE_AGGREGATION %s [partition_by=(%s), %s]", r.Table.Name(), partitionBy, props) + } + switch r.Grouping.Mode { + case types.GroupingModeByLabelSet, types.GroupingModeByEmptySet: + props = fmt.Sprintf("group_by=(%s), %s", grouping, props) + case types.GroupingModeWithoutLabelSet: + props = fmt.Sprintf("group_without=(%s), %s", grouping, props) } return fmt.Sprintf("RANGE_AGGREGATION %s [%s]", r.Table.Name(), props) diff --git a/pkg/engine/internal/planner/logical/node_vector_aggregate.go b/pkg/engine/internal/planner/logical/node_vector_aggregate.go index 79e00730e2bef..e37d76924fb73 100644 --- a/pkg/engine/internal/planner/logical/node_vector_aggregate.go +++ b/pkg/engine/internal/planner/logical/node_vector_aggregate.go @@ -14,8 +14,7 @@ type VectorAggregation struct { Table Value // The table relation to aggregate. - // The columns to group by. If empty, all rows are aggregated into a single result. - GroupBy []ColumnRef + Grouping Grouping // The grouping // The type of aggregation operation to perform (e.g., sum, min, max) Operation types.VectorAggregationType @@ -38,15 +37,20 @@ func (v *VectorAggregation) Name() string { func (v *VectorAggregation) String() string { props := fmt.Sprintf("operation=%s", v.Operation) - if len(v.GroupBy) > 0 { - groupBy := "" - for i, columnRef := range v.GroupBy { + grouping := "" + if len(v.Grouping.Columns) > 0 { + for i, columnRef := range v.Grouping.Columns { if i > 0 { - groupBy += ", " + grouping += ", " } - groupBy += columnRef.String() + grouping += columnRef.String() } - props += fmt.Sprintf(", group_by=(%s)", groupBy) + } + switch v.Grouping.Mode { + case types.GroupingModeByLabelSet, types.GroupingModeByEmptySet: + props = fmt.Sprintf("%s, group_by=(%s)", props, grouping) + case types.GroupingModeWithoutLabelSet: + props = fmt.Sprintf("%s, group_without=(%s)", props, grouping) } return fmt.Sprintf("VECTOR_AGGREGATION %s [%s]", v.Table.Name(), props) diff --git a/pkg/engine/internal/planner/logical/planner.go b/pkg/engine/internal/planner/logical/planner.go index 8ae35c2aa12d2..40901d3cd7ed3 100644 --- a/pkg/engine/internal/planner/logical/planner.go +++ b/pkg/engine/internal/planner/logical/planner.go @@ -274,7 +274,7 @@ func walkRangeAggregation(e *syntax.RangeAggregationExpr, params logql.Params) ( } builder = builder.RangeAggregation( - nil, rangeAggType, params.Start(), params.End(), params.Step(), rangeInterval, + convertGrouping(e.Grouping), rangeAggType, params.Start(), params.End(), params.Step(), rangeInterval, ) switch e.Operation { @@ -290,11 +290,6 @@ func walkRangeAggregation(e *syntax.RangeAggregationExpr, params logql.Params) ( } func walkVectorAggregation(e *syntax.VectorAggregationExpr, params logql.Params) (Value, error) { - // `without()` grouping is not supported. - if e.Grouping != nil && e.Grouping.Without { - return nil, errUnimplemented - } - left, err := walk(e.Left, params) if err != nil { return nil, err @@ -305,14 +300,9 @@ func walkVectorAggregation(e *syntax.VectorAggregationExpr, params logql.Params) return nil, errUnimplemented } - groupBy := make([]ColumnRef, 0, len(e.Grouping.Groups)) - for _, group := range e.Grouping.Groups { - groupBy = append(groupBy, *NewColumnRef(group, types.ColumnTypeAmbiguous)) - } - return &VectorAggregation{ Table: left, - GroupBy: groupBy, + Grouping: convertGrouping(e.Grouping), Operation: vecAggType, }, nil } @@ -431,7 +421,7 @@ func convertVectorAggregationType(op string) types.VectorAggregationType { switch op { case syntax.OpTypeSum: return types.VectorAggregationTypeSum - // case syntax.OpTypeCount: + //case syntax.OpTypeCount: // return types.VectorAggregationTypeCount case syntax.OpTypeMax: return types.VectorAggregationTypeMax @@ -597,6 +587,47 @@ func convertQueryRangeToPredicates(start, end time.Time) []*BinOp { } } +// convertGrouping converts [syntax.Grouping] structure into a list of columns and +// an explicit grouping mode. The way [syntax.Grouping] represents empty and non-empty +// label sets is fragile and does not survive protobuf encoding (nil vs empty slices), +// so an explicit [types.GroupingMode] is helpful. +func convertGrouping(g *syntax.Grouping) Grouping { + var grouping []ColumnRef + groupingMode := types.GroupingModeWithoutEmptySet + + if g == nil { + return Grouping{ + Columns: grouping, + Mode: groupingMode, + } + } + + if g.Groups != nil { + grouping = make([]ColumnRef, len(g.Groups)) + for i, group := range g.Groups { + grouping[i] = *NewColumnRef(group, types.ColumnTypeAmbiguous) + } + } + if g.Without { + if g.Groups != nil { + groupingMode = types.GroupingModeWithoutLabelSet + } else { + groupingMode = types.GroupingModeWithoutEmptySet + } + } else { + if g.Groups != nil { + groupingMode = types.GroupingModeByLabelSet + } else { + groupingMode = types.GroupingModeByEmptySet + } + } + + return Grouping{ + Columns: grouping, + Mode: groupingMode, + } +} + func parseShards(shards []string) (*ShardInfo, error) { if len(shards) == 0 { return noShard, nil diff --git a/pkg/engine/internal/planner/logical/planner_test.go b/pkg/engine/internal/planner/logical/planner_test.go index 9ecdab5fb3ffe..3e178c7a3a448 100644 --- a/pkg/engine/internal/planner/logical/planner_test.go +++ b/pkg/engine/internal/planner/logical/planner_test.go @@ -308,6 +308,7 @@ func TestCanExecuteQuery(t *testing.T) { }, { statement: `sum without (level) (count_over_time({env="prod"}[1m]))`, + expected: true, }, { // both vector and range aggregation are required @@ -317,10 +318,16 @@ func TestCanExecuteQuery(t *testing.T) { statement: `sum(count_over_time({env="prod"}[1m]))`, expected: true, }, + { + statement: `max(avg_over_time({env="prod"} | unwrap size [1m]))`, + }, { statement: `sum by (level) (rate({env="prod"}[1m]))`, expected: true, }, + { + statement: `avg by (level) (rate({env="prod"}[1m]))`, + }, { statement: `max by (level) (count_over_time({env="prod"}[1m]))`, expected: true, diff --git a/pkg/engine/internal/planner/physical/grouping.go b/pkg/engine/internal/planner/physical/grouping.go new file mode 100644 index 0000000000000..6149dc7e002e4 --- /dev/null +++ b/pkg/engine/internal/planner/physical/grouping.go @@ -0,0 +1,9 @@ +package physical + +import "github.com/grafana/loki/v3/pkg/engine/internal/types" + +// Grouping represents the grouping by/without label(s) for vector aggregators and range vector aggregators. +type Grouping struct { + Columns []ColumnExpression // The columns for grouping + Mode types.GroupingMode // The grouping mode +} diff --git a/pkg/engine/internal/planner/physical/optimizer.go b/pkg/engine/internal/planner/physical/optimizer.go index e488fa7e3080e..855dd0ae8d573 100644 --- a/pkg/engine/internal/planner/physical/optimizer.go +++ b/pkg/engine/internal/planner/physical/optimizer.go @@ -175,7 +175,9 @@ func (r *groupByPushdown) apply(root Node) bool { var changed bool for _, n := range nodes { vecAgg := n.(*VectorAggregation) - if len(vecAgg.GroupBy) == 0 { + + // Can only push down a non-empty by() label set + if vecAgg.Grouping.Mode != types.GroupingModeByLabelSet { continue } @@ -195,7 +197,7 @@ func (r *groupByPushdown) apply(root Node) bool { return false } - if r.applyToTargets(vecAgg, vecAgg.GroupBy, supportedAggTypes...) { + if r.applyToTargets(vecAgg, vecAgg.Grouping.Columns, supportedAggTypes...) { changed = true } } @@ -203,7 +205,7 @@ func (r *groupByPushdown) apply(root Node) bool { return changed } -func (r *groupByPushdown) applyToTargets(node Node, groupBy []ColumnExpression, supportedAggTypes ...types.RangeAggregationType) bool { +func (r *groupByPushdown) applyToTargets(node Node, grouping []ColumnExpression, supportedAggTypes ...types.RangeAggregationType) bool { var changed bool switch node := node.(type) { case *RangeAggregation: @@ -211,15 +213,21 @@ func (r *groupByPushdown) applyToTargets(node Node, groupBy []ColumnExpression, return false } - for _, colExpr := range groupBy { + // Cannot push down into without() + if node.Grouping.Mode == types.GroupingModeWithoutLabelSet { + return false + } + + for _, colExpr := range grouping { colExpr, ok := colExpr.(*ColumnExpr) if !ok { continue } var wasAdded bool - node.PartitionBy, wasAdded = addUniqueColumnExpr(node.PartitionBy, colExpr) + node.Grouping.Columns, wasAdded = addUniqueColumnExpr(node.Grouping.Columns, colExpr) if wasAdded { + node.Grouping.Mode = types.GroupingModeByLabelSet changed = true } } @@ -229,7 +237,7 @@ func (r *groupByPushdown) applyToTargets(node Node, groupBy []ColumnExpression, // Continue to children for _, child := range r.plan.Children(node) { - if r.applyToTargets(child, groupBy, supportedAggTypes...) { + if r.applyToTargets(child, grouping, supportedAggTypes...) { changed = true } } @@ -259,10 +267,12 @@ func (r *projectionPushdown) propagateProjections(node Node, projections []Colum var changed bool switch node := node.(type) { case *RangeAggregation: + if node.Grouping.Mode == types.GroupingModeWithoutEmptySet || node.Grouping.Mode == types.GroupingModeWithoutLabelSet { + return changed + } // [Source] RangeAggregation requires partitionBy columns & timestamp. - projections = append(projections, node.PartitionBy...) - // Always project timestamp column even if partitionBy is empty. - // Timestamp values are required to perform range aggregation. + projections = append(projections, node.Grouping.Columns...) + // Always project timestamp column. Timestamp values are required to perform range aggregation. projections = append(projections, &ColumnExpr{Ref: types.ColumnRef{Column: types.ColumnNameBuiltinTimestamp, Type: types.ColumnTypeBuiltin}}) case *Filter: // [Source] Filter nodes require predicate columns. diff --git a/pkg/engine/internal/planner/physical/optimizer_test.go b/pkg/engine/internal/planner/physical/optimizer_test.go index 2e9b777c8f2ca..a0a106aaabc5a 100644 --- a/pkg/engine/internal/planner/physical/optimizer_test.go +++ b/pkg/engine/internal/planner/physical/optimizer_test.go @@ -248,9 +248,12 @@ func TestLimitPushdown(t *testing.T) { func TestGroupByPushdown(t *testing.T) { t.Run("pushdown to RangeAggregation", func(t *testing.T) { - groupBy := []ColumnExpression{ - &ColumnExpr{Ref: types.ColumnRef{Column: "service", Type: types.ColumnTypeLabel}}, - &ColumnExpr{Ref: types.ColumnRef{Column: "level", Type: types.ColumnTypeLabel}}, + grouping := Grouping{ + Columns: []ColumnExpression{ + &ColumnExpr{Ref: types.ColumnRef{Column: "service", Type: types.ColumnTypeLabel}}, + &ColumnExpr{Ref: types.ColumnRef{Column: "level", Type: types.ColumnTypeLabel}}, + }, + Mode: types.GroupingModeByLabelSet, } // generate plan for sum by(service, instance) (count_over_time{...}[]) @@ -269,7 +272,7 @@ func TestGroupByPushdown(t *testing.T) { }) vectorAgg := plan.graph.Add(&VectorAggregation{ Operation: types.VectorAggregationTypeSum, - GroupBy: groupBy, + Grouping: grouping, }) _ = plan.graph.AddEdge(dag.Edge[Node]{Parent: vectorAgg, Child: rangeAgg}) @@ -295,12 +298,12 @@ func TestGroupByPushdown(t *testing.T) { Predicates: []Expression{}, }) rangeAgg := expectedPlan.graph.Add(&RangeAggregation{ - Operation: types.RangeAggregationTypeCount, - PartitionBy: groupBy, + Operation: types.RangeAggregationTypeCount, + Grouping: grouping, }) vectorAgg := expectedPlan.graph.Add(&VectorAggregation{ Operation: types.VectorAggregationTypeSum, - GroupBy: groupBy, + Grouping: grouping, }) _ = expectedPlan.graph.AddEdge(dag.Edge[Node]{Parent: vectorAgg, Child: rangeAgg}) @@ -313,8 +316,11 @@ func TestGroupByPushdown(t *testing.T) { }) t.Run("MAX->SUM is not allowed", func(t *testing.T) { - groupBy := []ColumnExpression{ - &ColumnExpr{Ref: types.ColumnRef{Column: "service", Type: types.ColumnTypeLabel}}, + grouping := Grouping{ + Columns: []ColumnExpression{ + &ColumnExpr{Ref: types.ColumnRef{Column: "service", Type: types.ColumnTypeLabel}}, + }, + Mode: types.GroupingModeByLabelSet, } // generate plan for max by(service) (sum_over_time{...}[]) @@ -332,7 +338,7 @@ func TestGroupByPushdown(t *testing.T) { }) vectorAgg := plan.graph.Add(&VectorAggregation{ Operation: types.VectorAggregationTypeMax, - GroupBy: groupBy, + Grouping: grouping, }) _ = plan.graph.AddEdge(dag.Edge[Node]{Parent: vectorAgg, Child: rangeAgg}) @@ -357,9 +363,12 @@ func TestGroupByPushdown(t *testing.T) { func TestProjectionPushdown(t *testing.T) { t.Run("range aggreagation groupBy -> scanset", func(t *testing.T) { - partitionBy := []ColumnExpression{ - &ColumnExpr{Ref: types.ColumnRef{Column: "level", Type: types.ColumnTypeLabel}}, - &ColumnExpr{Ref: types.ColumnRef{Column: "service", Type: types.ColumnTypeLabel}}, + grouping := Grouping{ + Columns: []ColumnExpression{ + &ColumnExpr{Ref: types.ColumnRef{Column: "level", Type: types.ColumnTypeLabel}}, + &ColumnExpr{Ref: types.ColumnRef{Column: "service", Type: types.ColumnTypeLabel}}, + }, + Mode: types.GroupingModeByLabelSet, } plan := &Plan{} @@ -371,8 +380,8 @@ func TestProjectionPushdown(t *testing.T) { }, }) rangeAgg := plan.graph.Add(&RangeAggregation{ - Operation: types.RangeAggregationTypeCount, - PartitionBy: partitionBy, + Operation: types.RangeAggregationTypeCount, + Grouping: grouping, }) _ = plan.graph.AddEdge(dag.Edge[Node]{Parent: rangeAgg, Child: scanset}) @@ -389,7 +398,7 @@ func TestProjectionPushdown(t *testing.T) { expectedPlan := &Plan{} { - projected := append(partitionBy, &ColumnExpr{Ref: types.ColumnRef{Column: types.ColumnNameBuiltinTimestamp, Type: types.ColumnTypeBuiltin}}) + projected := append(grouping.Columns, &ColumnExpr{Ref: types.ColumnRef{Column: types.ColumnNameBuiltinTimestamp, Type: types.ColumnTypeBuiltin}}) scanset := expectedPlan.graph.Add(&ScanSet{ Targets: []*ScanTarget{ {Type: ScanTypeDataObject, DataObject: &DataObjScan{}}, @@ -399,8 +408,8 @@ func TestProjectionPushdown(t *testing.T) { }) rangeAgg := expectedPlan.graph.Add(&RangeAggregation{ - Operation: types.RangeAggregationTypeCount, - PartitionBy: partitionBy, + Operation: types.RangeAggregationTypeCount, + Grouping: grouping, }) _ = expectedPlan.graph.AddEdge(dag.Edge[Node]{Parent: rangeAgg, Child: scanset}) @@ -653,22 +662,12 @@ func TestProjectionPushdown_PushesRequestedKeysToParseOperations(t *testing.T) { Op: types.BinaryOpEq, } builder = builder.Select(ambiguousFilter) - builder = builder.RangeAggregation( - nil, - types.RangeAggregationTypeCount, - time.Unix(0, 0), - time.Unix(3600, 0), - 5*time.Minute, - 5*time.Minute, - ) return builder.Value() }, - expectedParseKeysRequested: []string{"level"}, - expectedDataObjScanProjections: []string{"app", "level", "message", "timestamp"}, }, { - name: "RangeAggregation with PartitionBy on ambiguous columns", + name: "RangeAggregation with GroupBy on ambiguous columns", buildLogical: func() logical.Value { // count_over_time({app="test"} | logfmt [5m]) by (duration, service) builder := logical.NewBuilder(&logical.MakeTable{ @@ -682,11 +681,14 @@ func TestProjectionPushdown_PushesRequestedKeysToParseOperations(t *testing.T) { builder = builder.Parse(types.VariadicOpParseLogfmt, false, false) - // Range aggregation with PartitionBy + // Range aggregation with GroupBy builder = builder.RangeAggregation( - []logical.ColumnRef{ - {Ref: types.ColumnRef{Column: "duration", Type: types.ColumnTypeAmbiguous}}, - {Ref: types.ColumnRef{Column: "service", Type: types.ColumnTypeLabel}}, // Label should be skipped + logical.Grouping{ + Columns: []logical.ColumnRef{ + {Ref: types.ColumnRef{Column: "duration", Type: types.ColumnTypeAmbiguous}}, + {Ref: types.ColumnRef{Column: "service", Type: types.ColumnTypeLabel}}, // Label should be skipped + }, + Mode: types.GroupingModeByLabelSet, }, types.RangeAggregationTypeCount, time.Unix(0, 0), @@ -697,7 +699,7 @@ func TestProjectionPushdown_PushesRequestedKeysToParseOperations(t *testing.T) { return builder.Value() }, - expectedParseKeysRequested: []string{"duration"}, // Only ambiguous column from PartitionBy + expectedParseKeysRequested: []string{"duration"}, // Only ambiguous column from GroupBy expectedDataObjScanProjections: []string{"duration", "message", "service", "timestamp"}, }, { @@ -727,7 +729,7 @@ func TestProjectionPushdown_PushesRequestedKeysToParseOperations(t *testing.T) { // Range aggregation builder = builder.RangeAggregation( - []logical.ColumnRef{}, // no partition by + logical.NoGrouping, types.RangeAggregationTypeCount, time.Unix(0, 0), time.Unix(3600, 0), @@ -737,9 +739,12 @@ func TestProjectionPushdown_PushesRequestedKeysToParseOperations(t *testing.T) { // Vector aggregation with groupby on ambiguous columns builder = builder.VectorAggregation( - []logical.ColumnRef{ - {Ref: types.ColumnRef{Column: "status", Type: types.ColumnTypeAmbiguous}}, - {Ref: types.ColumnRef{Column: "code", Type: types.ColumnTypeAmbiguous}}, + logical.Grouping{ + Columns: []logical.ColumnRef{ + {Ref: types.ColumnRef{Column: "status", Type: types.ColumnTypeAmbiguous}}, + {Ref: types.ColumnRef{Column: "code", Type: types.ColumnTypeAmbiguous}}, + }, + Mode: types.GroupingModeByLabelSet, }, types.VectorAggregationTypeSum, ) diff --git a/pkg/engine/internal/planner/physical/planner.go b/pkg/engine/internal/planner/physical/planner.go index 0e01bf7f84971..5bb13a9ed468d 100644 --- a/pkg/engine/internal/planner/physical/planner.go +++ b/pkg/engine/internal/planner/physical/planner.go @@ -402,20 +402,23 @@ func (p *Planner) processLimit(lp *logical.Limit, ctx *Context) (Node, error) { } func (p *Planner) processRangeAggregation(r *logical.RangeAggregation, ctx *Context) (Node, error) { - partitionBy := make([]ColumnExpression, len(r.PartitionBy)) - for i, col := range r.PartitionBy { - partitionBy[i] = &ColumnExpr{Ref: col.Ref} + grouping := make([]ColumnExpression, len(r.Grouping.Columns)) + for i, col := range r.Grouping.Columns { + grouping[i] = &ColumnExpr{Ref: col.Ref} } node := &RangeAggregation{ NodeID: ulid.Make(), - PartitionBy: partitionBy, - Operation: r.Operation, - Start: r.Start, - End: r.End, - Range: r.RangeInterval, - Step: r.Step, + Grouping: Grouping{ + Columns: grouping, + Mode: r.Grouping.Mode, + }, + Operation: r.Operation, + Start: r.Start, + End: r.End, + Range: r.RangeInterval, + Step: r.Step, } p.plan.graph.Add(node) @@ -432,15 +435,18 @@ func (p *Planner) processRangeAggregation(r *logical.RangeAggregation, ctx *Cont // Convert [logical.VectorAggregation] into one [VectorAggregation] node. func (p *Planner) processVectorAggregation(lp *logical.VectorAggregation, ctx *Context) (Node, error) { - groupBy := make([]ColumnExpression, len(lp.GroupBy)) - for i, col := range lp.GroupBy { - groupBy[i] = &ColumnExpr{Ref: col.Ref} + grouping := make([]ColumnExpression, len(lp.Grouping.Columns)) + for i, col := range lp.Grouping.Columns { + grouping[i] = &ColumnExpr{Ref: col.Ref} } node := &VectorAggregation{ NodeID: ulid.Make(), - GroupBy: groupBy, + Grouping: Grouping{ + Columns: grouping, + Mode: lp.Grouping.Mode, + }, Operation: lp.Operation, } p.plan.graph.Add(node) diff --git a/pkg/engine/internal/planner/physical/planner_test.go b/pkg/engine/internal/planner/physical/planner_test.go index b7971e2601928..2310f6bb9a89b 100644 --- a/pkg/engine/internal/planner/physical/planner_test.go +++ b/pkg/engine/internal/planner/physical/planner_test.go @@ -361,7 +361,10 @@ func TestPlanner_Convert_WithParse(t *testing.T) { Op: types.BinaryOpEq, }, ).RangeAggregation( - []logical.ColumnRef{*logical.NewColumnRef("level", types.ColumnTypeAmbiguous)}, + logical.Grouping{ + Columns: []logical.ColumnRef{*logical.NewColumnRef("level", types.ColumnTypeAmbiguous)}, + Mode: types.GroupingModeByLabelSet, + }, types.RangeAggregationTypeCount, start, // Start time end, // End time @@ -511,7 +514,7 @@ func TestPlanner_Convert_RangeAggregations(t *testing.T) { Op: types.BinaryOpLt, }, ).RangeAggregation( - []logical.ColumnRef{}, + logical.NoGrouping, types.RangeAggregationTypeCount, time.Date(2023, 10, 1, 0, 0, 0, 0, time.UTC), // Start Time time.Date(2023, 10, 1, 1, 0, 0, 0, time.UTC), // End Time @@ -565,7 +568,7 @@ func TestPlanner_Convert_Rate(t *testing.T) { Op: types.BinaryOpLt, }, ).RangeAggregation( - []logical.ColumnRef{}, + logical.NoGrouping, types.RangeAggregationTypeCount, time.Date(2023, 10, 1, 0, 0, 0, 0, time.UTC), // Start Time time.Date(2023, 10, 1, 1, 0, 0, 0, time.UTC), // End Time @@ -621,7 +624,7 @@ func TestPlanner_BuildMathExpressions(t *testing.T) { Op: types.BinaryOpLt, }, ).RangeAggregation( - []logical.ColumnRef{}, + logical.NoGrouping, types.RangeAggregationTypeCount, time.Date(2023, 10, 1, 0, 0, 0, 0, time.UTC), // Start Time time.Date(2023, 10, 1, 1, 0, 0, 0, time.UTC), // End Time @@ -675,7 +678,7 @@ func TestPlanner_BuildMathExpressionsWithTwoInputs(t *testing.T) { Op: types.BinaryOpLt, }, ).RangeAggregation( - []logical.ColumnRef{}, + logical.NoGrouping, types.RangeAggregationTypeCount, time.Date(2023, 10, 1, 0, 0, 0, 0, time.UTC), // Start Time time.Date(2023, 10, 1, 1, 0, 0, 0, time.UTC), // End Time @@ -708,7 +711,7 @@ func TestPlanner_BuildMathExpressionsWithTwoInputs(t *testing.T) { Op: types.BinaryOpLt, }, ).RangeAggregation( - []logical.ColumnRef{}, + logical.NoGrouping, types.RangeAggregationTypeCount, time.Date(2023, 10, 1, 0, 0, 0, 0, time.UTC), // Start Time time.Date(2023, 10, 1, 1, 0, 0, 0, time.UTC), // End Time diff --git a/pkg/engine/internal/planner/physical/printer.go b/pkg/engine/internal/planner/physical/printer.go index 5f406c62a2cc9..53dc92f19ee47 100644 --- a/pkg/engine/internal/planner/physical/printer.go +++ b/pkg/engine/internal/planner/physical/printer.go @@ -6,6 +6,7 @@ import ( "strings" "time" + "github.com/grafana/loki/v3/pkg/engine/internal/types" "github.com/grafana/loki/v3/pkg/engine/internal/util/tree" ) @@ -63,7 +64,7 @@ func toTreeNode(n Node) *tree.Node { tree.NewProperty("limit", false, node.Fetch), } case *RangeAggregation: - properties := []tree.Property{ + treeNode.Properties = []tree.Property{ tree.NewProperty("operation", false, node.Operation), tree.NewProperty("start", false, node.Start.Format(time.RFC3339Nano)), tree.NewProperty("end", false, node.End.Format(time.RFC3339Nano)), @@ -71,19 +72,25 @@ func toTreeNode(n Node) *tree.Node { tree.NewProperty("range", false, node.Range), } - if len(node.PartitionBy) > 0 { - properties = append(properties, tree.NewProperty("partition_by", true, toAnySlice(node.PartitionBy)...)) + switch node.Grouping.Mode { + case types.GroupingModeByLabelSet, types.GroupingModeByEmptySet: + treeNode.Properties = append(treeNode.Properties, tree.NewProperty("group_by", true, toAnySlice(node.Grouping.Columns)...)) + case types.GroupingModeWithoutLabelSet: + treeNode.Properties = append(treeNode.Properties, tree.NewProperty("group_without", true, toAnySlice(node.Grouping.Columns)...)) } - - treeNode.Properties = properties case *VectorAggregation: - treeNode.Properties = []tree.Property{ + properties := []tree.Property{ tree.NewProperty("operation", false, node.Operation), } - if len(node.GroupBy) > 0 { - treeNode.Properties = append(treeNode.Properties, tree.NewProperty("group_by", true, toAnySlice(node.GroupBy)...)) + switch node.Grouping.Mode { + case types.GroupingModeByLabelSet, types.GroupingModeByEmptySet: + properties = append(properties, tree.NewProperty("group_by", true, toAnySlice(node.Grouping.Columns)...)) + case types.GroupingModeWithoutLabelSet: + properties = append(properties, tree.NewProperty("group_without", true, toAnySlice(node.Grouping.Columns)...)) } + + treeNode.Properties = properties case *ColumnCompat: treeNode.Properties = []tree.Property{ tree.NewProperty("src", false, node.Source), diff --git a/pkg/engine/internal/planner/physical/range_aggregate.go b/pkg/engine/internal/planner/physical/range_aggregate.go index c032575bb7cdc..d6b03d9bdb4c3 100644 --- a/pkg/engine/internal/planner/physical/range_aggregate.go +++ b/pkg/engine/internal/planner/physical/range_aggregate.go @@ -12,8 +12,7 @@ import ( type RangeAggregation struct { NodeID ulid.ULID - PartitionBy []ColumnExpression // Columns to partition the data by. - + Grouping Grouping Operation types.RangeAggregationType Start time.Time End time.Time @@ -29,8 +28,10 @@ func (r *RangeAggregation) Clone() Node { return &RangeAggregation{ NodeID: ulid.Make(), - PartitionBy: cloneExpressions(r.PartitionBy), - + Grouping: Grouping{ + Columns: cloneExpressions(r.Grouping.Columns), + Mode: r.Grouping.Mode, + }, Operation: r.Operation, Start: r.Start, End: r.End, diff --git a/pkg/engine/internal/planner/physical/vector_aggregate.go b/pkg/engine/internal/planner/physical/vector_aggregate.go index 2448601e76553..0622dde2861b9 100644 --- a/pkg/engine/internal/planner/physical/vector_aggregate.go +++ b/pkg/engine/internal/planner/physical/vector_aggregate.go @@ -12,8 +12,7 @@ import ( type VectorAggregation struct { NodeID ulid.ULID - // GroupBy defines the columns to group by. If empty, all rows are aggregated into a single result. - GroupBy []ColumnExpression + Grouping Grouping // Grouping of the data. // Operation defines the type of aggregation operation to perform (e.g., sum, min, max) Operation types.VectorAggregationType @@ -28,7 +27,10 @@ func (v *VectorAggregation) Clone() Node { return &VectorAggregation{ NodeID: ulid.Make(), - GroupBy: cloneExpressions(v.GroupBy), + Grouping: Grouping{ + Columns: cloneExpressions(v.Grouping.Columns), + Mode: v.Grouping.Mode, + }, Operation: v.Operation, } } diff --git a/pkg/engine/internal/planner/planner_test.go b/pkg/engine/internal/planner/planner_test.go index 90a00dc2d4324..b912d5f728dcf 100644 --- a/pkg/engine/internal/planner/planner_test.go +++ b/pkg/engine/internal/planner/planner_test.go @@ -201,7 +201,7 @@ TopK sort_by=builtin.timestamp ascending=false nulls_first=false k=1000 query: `sum by (bar) (sum_over_time({app="foo"} | logfmt | request_duration != "" | unwrap duration(request_duration)[1m]))`, expected: ` VectorAggregation operation=sum group_by=(ambiguous.bar) -└── RangeAggregation operation=sum start=2025-01-01T00:00:00Z end=2025-01-01T01:00:00Z step=0s range=1m0s partition_by=(ambiguous.bar) +└── RangeAggregation operation=sum start=2025-01-01T00:00:00Z end=2025-01-01T01:00:00Z step=0s range=1m0s group_by=(ambiguous.bar) └── Parallelize └── Projection all=true expand=(CAST_DURATION(ambiguous.request_duration)) └── Filter predicate[0]=NEQ(ambiguous.request_duration, "") @@ -217,7 +217,7 @@ VectorAggregation operation=sum group_by=(ambiguous.bar) comment: `metric: multiple parse stages`, query: `sum(count_over_time({app="foo"} | detected_level="error" | json | logfmt | drop __error__,__error_details__[1m]))`, expected: ` -VectorAggregation operation=sum +VectorAggregation operation=sum group_by=() └── RangeAggregation operation=count start=2025-01-01T00:00:00Z end=2025-01-01T01:00:00Z step=0s range=1m0s └── Parallelize └── Projection all=true drop=(ambiguous.__error__, ambiguous.__error_details__) @@ -227,7 +227,7 @@ VectorAggregation operation=sum └── Projection all=true expand=(PARSE_LOGFMT(builtin.message, [], false, false)) └── Filter predicate[0]=EQ(ambiguous.detected_level, "error") └── Compat src=metadata dst=metadata collision=label - └── ScanSet num_targets=2 projections=(ambiguous.detected_level, builtin.message, builtin.timestamp) predicate[0]=GTE(builtin.timestamp, 2024-12-31T23:59:00Z) predicate[1]=LT(builtin.timestamp, 2025-01-01T01:00:00Z) + └── ScanSet num_targets=2 predicate[0]=GTE(builtin.timestamp, 2024-12-31T23:59:00Z) predicate[1]=LT(builtin.timestamp, 2025-01-01T01:00:00Z) ├── @target type=ScanTypeDataObject location=objects/00/0000000000.dataobj streams=5 section_id=1 projections=() └── @target type=ScanTypeDataObject location=objects/00/0000000000.dataobj streams=5 section_id=0 projections=() @@ -239,7 +239,7 @@ VectorAggregation operation=sum expected: ` VectorAggregation operation=sum group_by=(ambiguous.bar) └── Projection all=true expand=(DIV(generated.value, 300)) - └── RangeAggregation operation=count start=2025-01-01T00:00:00Z end=2025-01-01T01:00:00Z step=0s range=1m0s partition_by=(ambiguous.bar) + └── RangeAggregation operation=count start=2025-01-01T00:00:00Z end=2025-01-01T01:00:00Z step=0s range=1m0s group_by=(ambiguous.bar) └── Parallelize └── Compat src=metadata dst=metadata collision=label └── ScanSet num_targets=2 projections=(ambiguous.bar, builtin.timestamp) predicate[0]=GTE(builtin.timestamp, 2024-12-31T23:59:00Z) predicate[1]=LT(builtin.timestamp, 2025-01-01T01:00:00Z) @@ -267,7 +267,7 @@ TopK sort_by=builtin.timestamp ascending=false nulls_first=false k=1000 query: `sum by (bar) (count_over_time({app="foo"} | logfmt[1m]))`, expected: ` VectorAggregation operation=sum group_by=(ambiguous.bar) -└── RangeAggregation operation=count start=2025-01-01T00:00:00Z end=2025-01-01T01:00:00Z step=0s range=1m0s partition_by=(ambiguous.bar) +└── RangeAggregation operation=count start=2025-01-01T00:00:00Z end=2025-01-01T01:00:00Z step=0s range=1m0s group_by=(ambiguous.bar) └── Parallelize └── Compat src=parsed dst=parsed collision=label └── Projection all=true expand=(PARSE_LOGFMT(builtin.message, [bar], false, false)) diff --git a/pkg/engine/internal/proto/physicalpb/marshal_node.go b/pkg/engine/internal/proto/physicalpb/marshal_node.go index ccb733924dcf6..250d5cd2b5006 100644 --- a/pkg/engine/internal/proto/physicalpb/marshal_node.go +++ b/pkg/engine/internal/proto/physicalpb/marshal_node.go @@ -97,11 +97,15 @@ func (n *AggregateRange) MarshalPhysical(nodeID ulid.ULID) (physical.Node, error return nil, err } + grouping, err := marshalGrouping(n.Grouping) + if err != nil { + return nil, err + } + return &physical.RangeAggregation{ NodeID: nodeID, - PartitionBy: marshalColumnExpressions(n.PartitionBy), - + Grouping: grouping, Operation: operation, Start: n.Start, End: n.End, @@ -110,6 +114,22 @@ func (n *AggregateRange) MarshalPhysical(nodeID ulid.ULID) (physical.Node, error }, nil } +func marshalGrouping(g *Grouping) (physical.Grouping, error) { + if g == nil { + return physical.Grouping{}, fmt.Errorf("empty grouping") + } + + mode, err := g.Mode.marshalType() + if err != nil { + return physical.Grouping{}, err + } + + return physical.Grouping{ + Columns: marshalColumnExpressions(g.Columns), + Mode: mode, + }, nil +} + func marshalColumnExpressions(exprs []*expressionpb.ColumnExpression) []physical.ColumnExpression { if exprs == nil { return nil @@ -134,10 +154,15 @@ func (n *AggregateVector) MarshalPhysical(nodeID ulid.ULID) (physical.Node, erro return nil, err } + grouping, err := marshalGrouping(n.Grouping) + if err != nil { + return nil, err + } + return &physical.VectorAggregation{ NodeID: nodeID, - GroupBy: marshalColumnExpressions(n.GroupBy), + Grouping: grouping, Operation: operation, }, nil } diff --git a/pkg/engine/internal/proto/physicalpb/marshal_types.go b/pkg/engine/internal/proto/physicalpb/marshal_types.go index b60139281e0a0..6a28c0c1e789c 100644 --- a/pkg/engine/internal/proto/physicalpb/marshal_types.go +++ b/pkg/engine/internal/proto/physicalpb/marshal_types.go @@ -30,6 +30,13 @@ var ( AGGREGATE_VECTOR_OP_SORT: types.VectorAggregationTypeSort, AGGREGATE_VECTOR_OP_SORT_DESC: types.VectorAggregationTypeSortDesc, } + + nativeGroupingModeLookup = map[GroupingMode]types.GroupingMode{ + GROUPING_MODE_GROUPING_BY_EMPTY_SET: types.GroupingModeByEmptySet, + GROUPING_MODE_GROUPING_BY_LABEL_SET: types.GroupingModeByLabelSet, + GROUPING_MODE_GROUPING_WITHOUT_EMPTY_SET: types.GroupingModeWithoutEmptySet, + GROUPING_MODE_GROUPING_WITHOUT_LABEL_SET: types.GroupingModeWithoutLabelSet, + } ) func (op AggregateRangeOp) marshalType() (types.RangeAggregationType, error) { @@ -45,3 +52,10 @@ func (op AggregateVectorOp) marshalType() (types.VectorAggregationType, error) { } return types.VectorAggregationTypeInvalid, fmt.Errorf("unknown AggregateVectorOp: %v", op) } + +func (m GroupingMode) marshalType() (types.GroupingMode, error) { + if result, ok := nativeGroupingModeLookup[m]; ok { + return result, nil + } + return types.GroupingModeInvalid, fmt.Errorf("unknown GroupingMode: %v", m) +} diff --git a/pkg/engine/internal/proto/physicalpb/physicalpb.pb.go b/pkg/engine/internal/proto/physicalpb/physicalpb.pb.go index 202c24810a3b6..d6a36b37ba901 100644 --- a/pkg/engine/internal/proto/physicalpb/physicalpb.pb.go +++ b/pkg/engine/internal/proto/physicalpb/physicalpb.pb.go @@ -34,6 +34,37 @@ var _ = time.Kitchen // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +// GroupingMode represents the way on interpretation of the list of columns for grouping. +type GroupingMode int32 + +const ( + GROUPING_MODE_INVALID GroupingMode = 0 + GROUPING_MODE_GROUPING_BY_EMPTY_SET GroupingMode = 1 + GROUPING_MODE_GROUPING_BY_LABEL_SET GroupingMode = 2 + GROUPING_MODE_GROUPING_WITHOUT_EMPTY_SET GroupingMode = 3 + GROUPING_MODE_GROUPING_WITHOUT_LABEL_SET GroupingMode = 4 +) + +var GroupingMode_name = map[int32]string{ + 0: "GROUPING_MODE_INVALID", + 1: "GROUPING_MODE_GROUPING_BY_EMPTY_SET", + 2: "GROUPING_MODE_GROUPING_BY_LABEL_SET", + 3: "GROUPING_MODE_GROUPING_WITHOUT_EMPTY_SET", + 4: "GROUPING_MODE_GROUPING_WITHOUT_LABEL_SET", +} + +var GroupingMode_value = map[string]int32{ + "GROUPING_MODE_INVALID": 0, + "GROUPING_MODE_GROUPING_BY_EMPTY_SET": 1, + "GROUPING_MODE_GROUPING_BY_LABEL_SET": 2, + "GROUPING_MODE_GROUPING_WITHOUT_EMPTY_SET": 3, + "GROUPING_MODE_GROUPING_WITHOUT_LABEL_SET": 4, +} + +func (GroupingMode) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_d1fdbb44b95b211f, []int{0} +} + // AggregateRangeOp represents the operation to perform on the aggregated // data. type AggregateRangeOp int32 @@ -66,7 +97,7 @@ var AggregateRangeOp_value = map[string]int32{ } func (AggregateRangeOp) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{0} + return fileDescriptor_d1fdbb44b95b211f, []int{1} } // AggregateVectorOp represents the different aggregation operations that can @@ -119,7 +150,7 @@ var AggregateVectorOp_value = map[string]int32{ } func (AggregateVectorOp) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{1} + return fileDescriptor_d1fdbb44b95b211f, []int{2} } // SortOrder represents the order in which results should be sorted. @@ -144,7 +175,7 @@ var SortOrder_value = map[string]int32{ } func (SortOrder) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{2} + return fileDescriptor_d1fdbb44b95b211f, []int{3} } // Plan represents the physical plan of a query. @@ -502,13 +533,13 @@ func (*Node) XXX_OneofWrappers() []interface{} { // AggregateRange aggregates samples into windowed ranges. type AggregateRange struct { - // Columns to partition the data by. - PartitionBy []*expressionpb.ColumnExpression `protobuf:"bytes,1,rep,name=partition_by,json=partitionBy,proto3" json:"partition_by,omitempty"` - Operation AggregateRangeOp `protobuf:"varint,2,opt,name=operation,proto3,enum=loki.physical.AggregateRangeOp" json:"operation,omitempty"` - Start time.Time `protobuf:"bytes,3,opt,name=start,proto3,stdtime" json:"start"` - End time.Time `protobuf:"bytes,4,opt,name=end,proto3,stdtime" json:"end"` - Step time.Duration `protobuf:"bytes,5,opt,name=step,proto3,stdduration" json:"step"` - Range time.Duration `protobuf:"bytes,6,opt,name=range,proto3,stdduration" json:"range"` + // Grouping for the data. + Grouping *Grouping `protobuf:"bytes,1,opt,name=grouping,proto3" json:"grouping,omitempty"` + Operation AggregateRangeOp `protobuf:"varint,2,opt,name=operation,proto3,enum=loki.physical.AggregateRangeOp" json:"operation,omitempty"` + Start time.Time `protobuf:"bytes,3,opt,name=start,proto3,stdtime" json:"start"` + End time.Time `protobuf:"bytes,4,opt,name=end,proto3,stdtime" json:"end"` + Step time.Duration `protobuf:"bytes,5,opt,name=step,proto3,stdduration" json:"step"` + Range time.Duration `protobuf:"bytes,6,opt,name=range,proto3,stdduration" json:"range"` } func (m *AggregateRange) Reset() { *m = AggregateRange{} } @@ -543,9 +574,9 @@ func (m *AggregateRange) XXX_DiscardUnknown() { var xxx_messageInfo_AggregateRange proto.InternalMessageInfo -func (m *AggregateRange) GetPartitionBy() []*expressionpb.ColumnExpression { +func (m *AggregateRange) GetGrouping() *Grouping { if m != nil { - return m.PartitionBy + return m.Grouping } return nil } @@ -585,19 +616,72 @@ func (m *AggregateRange) GetRange() time.Duration { return 0 } +// Grouping represents the grouping by/without label(s) for vector aggregators +// and range vector aggregators. +type Grouping struct { + Columns []*expressionpb.ColumnExpression `protobuf:"bytes,1,rep,name=columns,proto3" json:"columns,omitempty"` + Mode GroupingMode `protobuf:"varint,2,opt,name=mode,proto3,enum=loki.physical.GroupingMode" json:"mode,omitempty"` +} + +func (m *Grouping) Reset() { *m = Grouping{} } +func (*Grouping) ProtoMessage() {} +func (*Grouping) Descriptor() ([]byte, []int) { + return fileDescriptor_d1fdbb44b95b211f, []int{5} +} +func (m *Grouping) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Grouping) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Grouping.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Grouping) XXX_Merge(src proto.Message) { + xxx_messageInfo_Grouping.Merge(m, src) +} +func (m *Grouping) XXX_Size() int { + return m.Size() +} +func (m *Grouping) XXX_DiscardUnknown() { + xxx_messageInfo_Grouping.DiscardUnknown(m) +} + +var xxx_messageInfo_Grouping proto.InternalMessageInfo + +func (m *Grouping) GetColumns() []*expressionpb.ColumnExpression { + if m != nil { + return m.Columns + } + return nil +} + +func (m *Grouping) GetMode() GroupingMode { + if m != nil { + return m.Mode + } + return GROUPING_MODE_INVALID +} + // AggregateVector represents an operation to aggregate a range vector into an // instant vector, with optional grouping on specified dimensions. type AggregateVector struct { - // Columns to group the data by. If empty, all rows are aggregated into a single result. - GroupBy []*expressionpb.ColumnExpression `protobuf:"bytes,1,rep,name=group_by,json=groupBy,proto3" json:"group_by,omitempty"` + // Grouping for the data. + Grouping *Grouping `protobuf:"bytes,1,opt,name=grouping,proto3" json:"grouping,omitempty"` // Aggregation operation to perform over the underlying range vector. - Operation AggregateVectorOp `protobuf:"varint,2,opt,name=operation,proto3,enum=loki.physical.AggregateVectorOp" json:"operation,omitempty"` + Operation AggregateVectorOp `protobuf:"varint,3,opt,name=operation,proto3,enum=loki.physical.AggregateVectorOp" json:"operation,omitempty"` } func (m *AggregateVector) Reset() { *m = AggregateVector{} } func (*AggregateVector) ProtoMessage() {} func (*AggregateVector) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{5} + return fileDescriptor_d1fdbb44b95b211f, []int{6} } func (m *AggregateVector) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -626,9 +710,9 @@ func (m *AggregateVector) XXX_DiscardUnknown() { var xxx_messageInfo_AggregateVector proto.InternalMessageInfo -func (m *AggregateVector) GetGroupBy() []*expressionpb.ColumnExpression { +func (m *AggregateVector) GetGrouping() *Grouping { if m != nil { - return m.GroupBy + return m.Grouping } return nil } @@ -660,7 +744,7 @@ type DataObjScan struct { func (m *DataObjScan) Reset() { *m = DataObjScan{} } func (*DataObjScan) ProtoMessage() {} func (*DataObjScan) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{6} + return fileDescriptor_d1fdbb44b95b211f, []int{7} } func (m *DataObjScan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -739,7 +823,7 @@ type TimeRange struct { func (m *TimeRange) Reset() { *m = TimeRange{} } func (*TimeRange) ProtoMessage() {} func (*TimeRange) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{7} + return fileDescriptor_d1fdbb44b95b211f, []int{8} } func (m *TimeRange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -791,7 +875,7 @@ type Filter struct { func (m *Filter) Reset() { *m = Filter{} } func (*Filter) ProtoMessage() {} func (*Filter) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{8} + return fileDescriptor_d1fdbb44b95b211f, []int{9} } func (m *Filter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -839,7 +923,7 @@ type Limit struct { func (m *Limit) Reset() { *m = Limit{} } func (*Limit) ProtoMessage() {} func (*Limit) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{9} + return fileDescriptor_d1fdbb44b95b211f, []int{10} } func (m *Limit) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -897,7 +981,7 @@ type Projection struct { func (m *Projection) Reset() { *m = Projection{} } func (*Projection) ProtoMessage() {} func (*Projection) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{10} + return fileDescriptor_d1fdbb44b95b211f, []int{11} } func (m *Projection) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -965,7 +1049,7 @@ type ColumnCompat struct { func (m *ColumnCompat) Reset() { *m = ColumnCompat{} } func (*ColumnCompat) ProtoMessage() {} func (*ColumnCompat) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{11} + return fileDescriptor_d1fdbb44b95b211f, []int{12} } func (m *ColumnCompat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1028,7 +1112,7 @@ type TopK struct { func (m *TopK) Reset() { *m = TopK{} } func (*TopK) ProtoMessage() {} func (*TopK) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{12} + return fileDescriptor_d1fdbb44b95b211f, []int{13} } func (m *TopK) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1102,7 +1186,7 @@ type ScanSet struct { func (m *ScanSet) Reset() { *m = ScanSet{} } func (*ScanSet) ProtoMessage() {} func (*ScanSet) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{13} + return fileDescriptor_d1fdbb44b95b211f, []int{14} } func (m *ScanSet) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1163,7 +1247,7 @@ type ScanTarget struct { func (m *ScanTarget) Reset() { *m = ScanTarget{} } func (*ScanTarget) ProtoMessage() {} func (*ScanTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{14} + return fileDescriptor_d1fdbb44b95b211f, []int{15} } func (m *ScanTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1235,7 +1319,7 @@ type Parallelize struct { func (m *Parallelize) Reset() { *m = Parallelize{} } func (*Parallelize) ProtoMessage() {} func (*Parallelize) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{15} + return fileDescriptor_d1fdbb44b95b211f, []int{16} } func (m *Parallelize) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1272,7 +1356,7 @@ type Join struct { func (m *Join) Reset() { *m = Join{} } func (*Join) ProtoMessage() {} func (*Join) Descriptor() ([]byte, []int) { - return fileDescriptor_d1fdbb44b95b211f, []int{16} + return fileDescriptor_d1fdbb44b95b211f, []int{17} } func (m *Join) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1302,6 +1386,7 @@ func (m *Join) XXX_DiscardUnknown() { var xxx_messageInfo_Join proto.InternalMessageInfo func init() { + proto.RegisterEnum("loki.physical.GroupingMode", GroupingMode_name, GroupingMode_value) proto.RegisterEnum("loki.physical.AggregateRangeOp", AggregateRangeOp_name, AggregateRangeOp_value) proto.RegisterEnum("loki.physical.AggregateVectorOp", AggregateVectorOp_name, AggregateVectorOp_value) proto.RegisterEnum("loki.physical.SortOrder", SortOrder_name, SortOrder_value) @@ -1310,6 +1395,7 @@ func init() { proto.RegisterType((*NodeID)(nil), "loki.physical.NodeID") proto.RegisterType((*Node)(nil), "loki.physical.Node") proto.RegisterType((*AggregateRange)(nil), "loki.physical.AggregateRange") + proto.RegisterType((*Grouping)(nil), "loki.physical.Grouping") proto.RegisterType((*AggregateVector)(nil), "loki.physical.AggregateVector") proto.RegisterType((*DataObjScan)(nil), "loki.physical.DataObjScan") proto.RegisterType((*TimeRange)(nil), "loki.physical.TimeRange") @@ -1329,104 +1415,118 @@ func init() { } var fileDescriptor_d1fdbb44b95b211f = []byte{ - // 1516 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0x4f, 0x6f, 0xdb, 0xc8, - 0x15, 0x27, 0xf5, 0x5f, 0x4f, 0xb6, 0x97, 0x9d, 0x66, 0x13, 0xda, 0xd9, 0xd0, 0x59, 0x9d, 0x12, - 0xb7, 0x95, 0x9a, 0x18, 0x6d, 0x91, 0x6d, 0xd7, 0x85, 0xfe, 0xc5, 0x52, 0x9d, 0x48, 0xc6, 0x48, - 0x31, 0xb6, 0xbd, 0xa8, 0x23, 0x72, 0x4c, 0x33, 0xa6, 0x48, 0x82, 0xa4, 0x0c, 0xbb, 0x97, 0xed, - 0xad, 0xe8, 0xa9, 0x7b, 0xec, 0x07, 0xe8, 0xa1, 0xc7, 0x7e, 0x81, 0x02, 0x45, 0x4f, 0x7b, 0xcc, - 0x71, 0xd1, 0xc3, 0xb6, 0x71, 0x2e, 0x3d, 0xee, 0x17, 0x28, 0x50, 0xcc, 0x0c, 0x25, 0x52, 0xb2, - 0xe4, 0xb5, 0x83, 0xbd, 0x18, 0x33, 0xef, 0xfd, 0x7e, 0x6f, 0xde, 0x7b, 0x7c, 0x7f, 0x2c, 0xf8, - 0x89, 0x77, 0x6a, 0x56, 0xa9, 0x63, 0x5a, 0x0e, 0xad, 0x5a, 0x4e, 0x48, 0x7d, 0x87, 0xd8, 0x55, - 0xcf, 0x77, 0x43, 0xb7, 0xea, 0x9d, 0x5c, 0x04, 0x96, 0x4e, 0x6c, 0x6f, 0x94, 0x38, 0x56, 0xb8, - 0x0e, 0xad, 0xdb, 0xee, 0xa9, 0x55, 0x99, 0x8a, 0xb7, 0xee, 0x98, 0xae, 0xe9, 0x0a, 0x16, 0x3b, - 0x09, 0xd0, 0x96, 0x66, 0xba, 0xae, 0x69, 0x53, 0x61, 0x6e, 0x34, 0x39, 0xae, 0x1a, 0x13, 0x9f, - 0x84, 0x96, 0xeb, 0x44, 0xfa, 0xed, 0x45, 0x7d, 0x68, 0x8d, 0x69, 0x10, 0x92, 0xb1, 0x17, 0x01, - 0x9e, 0xad, 0x76, 0x8e, 0x9e, 0x7b, 0x3e, 0x0d, 0x02, 0xcb, 0x75, 0xbc, 0xd1, 0xdc, 0x25, 0xa2, - 0x3e, 0x5e, 0x4d, 0x9d, 0xd8, 0x96, 0xc1, 0xff, 0x08, 0x68, 0xf9, 0xb7, 0x90, 0x39, 0xb4, 0x89, - 0x83, 0x1e, 0x43, 0xd6, 0x71, 0x0d, 0x1a, 0xa8, 0xf2, 0xc3, 0xf4, 0xa3, 0xd2, 0xd3, 0xef, 0x57, - 0xe6, 0x62, 0xac, 0x74, 0x5d, 0x83, 0x62, 0x81, 0x40, 0x3f, 0x82, 0x2c, 0x35, 0x4c, 0x1a, 0xa8, - 0x29, 0x0e, 0xbd, 0xb7, 0x00, 0x65, 0xe6, 0x5a, 0x86, 0x49, 0xb1, 0x40, 0x95, 0x7d, 0x28, 0x4c, - 0x45, 0x68, 0x17, 0x72, 0x1e, 0xf1, 0xa9, 0x13, 0xaa, 0xf2, 0x43, 0xf9, 0x51, 0xe9, 0xe9, 0x87, - 0x4b, 0x9e, 0xe9, 0x34, 0xeb, 0x99, 0x2f, 0xbf, 0xde, 0x96, 0x70, 0x04, 0x45, 0x4f, 0x20, 0xab, - 0x9f, 0x58, 0xb6, 0xa1, 0xa6, 0xbe, 0x9d, 0x23, 0x90, 0xe5, 0xcf, 0x21, 0x27, 0xc4, 0x68, 0x02, - 0xd9, 0x33, 0x62, 0x4f, 0x68, 0xf4, 0xe0, 0xbd, 0x0a, 0x8f, 0x9d, 0x5b, 0x38, 0x7b, 0x52, 0x39, - 0x64, 0x39, 0x78, 0xf5, 0xa2, 0xd3, 0xac, 0x3f, 0x67, 0xf4, 0x7f, 0x7d, 0xbd, 0xbd, 0x67, 0x5a, - 0xe1, 0xc9, 0x64, 0x54, 0xd1, 0xdd, 0x71, 0xd5, 0xf4, 0xc9, 0x31, 0x71, 0x48, 0x95, 0xa1, 0xab, - 0x67, 0xbb, 0xd5, 0xeb, 0xb3, 0x5a, 0x61, 0x76, 0xb0, 0x78, 0xad, 0xfc, 0xb7, 0x2c, 0x64, 0x98, - 0x07, 0xe8, 0x07, 0x90, 0xb2, 0x8c, 0x9b, 0x44, 0x9b, 0xb2, 0x0c, 0xd4, 0x86, 0x0f, 0x88, 0x69, - 0xfa, 0xd4, 0x24, 0x21, 0x1d, 0xfa, 0xc4, 0x31, 0x69, 0x14, 0xf3, 0x83, 0x05, 0x66, 0x6d, 0x8a, - 0xc2, 0x0c, 0xd4, 0x96, 0xf0, 0x06, 0x99, 0x93, 0xa0, 0x03, 0x50, 0x62, 0x4b, 0x67, 0x54, 0x0f, - 0x5d, 0x5f, 0x4d, 0x73, 0x53, 0xda, 0x2a, 0x53, 0x47, 0x1c, 0xd5, 0x96, 0x70, 0xec, 0x83, 0x10, - 0xa1, 0x1f, 0x43, 0x26, 0xd0, 0x89, 0xa3, 0x66, 0xb8, 0x81, 0xad, 0x05, 0x03, 0x4d, 0x12, 0x92, - 0xde, 0xe8, 0x75, 0x5f, 0x27, 0x4e, 0x5b, 0xc2, 0x1c, 0x89, 0xaa, 0x90, 0x3b, 0xb6, 0xec, 0x90, - 0xfa, 0x6a, 0x76, 0x69, 0xe4, 0xcf, 0xb9, 0xb2, 0x2d, 0xe1, 0x08, 0x86, 0x7e, 0x08, 0x59, 0xdb, - 0x1a, 0x5b, 0xa1, 0x9a, 0xe3, 0xf8, 0x3b, 0x0b, 0xf8, 0x17, 0x4c, 0xd7, 0x96, 0xb0, 0x00, 0xa1, - 0x9f, 0x03, 0x78, 0xbe, 0xfb, 0x9a, 0xea, 0xac, 0x9f, 0xd4, 0x3c, 0xa7, 0x6c, 0x2e, 0x96, 0xe1, - 0x0c, 0xd0, 0x96, 0x70, 0x02, 0x8e, 0xea, 0xb0, 0xae, 0xbb, 0xf6, 0x64, 0xec, 0x0c, 0x75, 0x77, - 0xec, 0x91, 0x50, 0x2d, 0x70, 0xfe, 0xfd, 0x05, 0x7e, 0x83, 0x63, 0x1a, 0x1c, 0xd2, 0x96, 0xf0, - 0x9a, 0x9e, 0xb8, 0xa3, 0x5d, 0x28, 0xb0, 0x38, 0x87, 0x01, 0x0d, 0xd5, 0x22, 0xa7, 0xdf, 0x5d, - 0xa0, 0xb3, 0x74, 0xf4, 0x29, 0x63, 0xe6, 0x03, 0x71, 0x44, 0x3b, 0x90, 0x0d, 0x5d, 0x6f, 0x78, - 0xaa, 0x02, 0x67, 0x2c, 0xb6, 0xd8, 0xc0, 0xf5, 0x0e, 0x58, 0x02, 0x43, 0xd7, 0x3b, 0x40, 0x7b, - 0x50, 0xf2, 0x88, 0x4f, 0x6c, 0x9b, 0xda, 0xd6, 0xef, 0xa8, 0x5a, 0x5a, 0x9a, 0xf9, 0xc3, 0x18, - 0xd1, 0x96, 0x70, 0x92, 0x80, 0x1e, 0x43, 0xe6, 0xb5, 0x6b, 0x39, 0xea, 0xda, 0xd2, 0xa7, 0x7e, - 0xe5, 0x5a, 0xfc, 0x5b, 0x31, 0x48, 0x3d, 0x07, 0x99, 0x53, 0xcb, 0x31, 0xca, 0xff, 0x4b, 0xc1, - 0xc6, 0x7c, 0x5d, 0xa1, 0x26, 0xac, 0x79, 0xc4, 0x0f, 0x2d, 0x96, 0xb7, 0xe1, 0xe8, 0x22, 0x9a, - 0x0d, 0x1f, 0x0b, 0x6b, 0xf1, 0xdc, 0x89, 0x72, 0xd5, 0x9a, 0x09, 0xb8, 0x2f, 0x82, 0x56, 0xbf, - 0x40, 0x9f, 0x42, 0xd1, 0xf5, 0xa8, 0x18, 0x7e, 0xbc, 0x9e, 0x37, 0x9e, 0x6e, 0x5f, 0x5b, 0xcf, - 0x3d, 0x0f, 0xc7, 0x0c, 0xf4, 0x09, 0x64, 0x83, 0x90, 0xf8, 0x61, 0x54, 0xbf, 0x5b, 0x15, 0x31, - 0x38, 0x2b, 0xd3, 0xc1, 0x59, 0x19, 0x4c, 0x07, 0x67, 0xbd, 0xc0, 0x3a, 0xe9, 0x8b, 0x7f, 0x6f, - 0xcb, 0x58, 0x50, 0xd0, 0x4f, 0x21, 0x4d, 0x1d, 0x63, 0x56, 0xb8, 0x37, 0x61, 0x32, 0x02, 0xfa, - 0x19, 0x64, 0x82, 0x90, 0x7a, 0x51, 0xf5, 0x6e, 0x5e, 0x21, 0x36, 0xa3, 0x59, 0x2e, 0x78, 0x7f, - 0x66, 0x3c, 0x4e, 0x40, 0xcf, 0x20, 0x2b, 0xfa, 0x36, 0x77, 0x73, 0xa6, 0x60, 0x94, 0xff, 0x24, - 0xc3, 0x07, 0x0b, 0xcd, 0x88, 0x7e, 0x01, 0x05, 0xd3, 0x77, 0x27, 0xde, 0xad, 0x92, 0x9f, 0xe7, - 0x94, 0xfa, 0x05, 0xda, 0xbb, 0x9a, 0xf8, 0x87, 0xd7, 0x77, 0xff, 0x5c, 0xe6, 0xcb, 0x7f, 0x49, - 0x41, 0x29, 0xd1, 0xdd, 0x68, 0x0b, 0x0a, 0xb6, 0xab, 0x0b, 0x73, 0x6c, 0xa2, 0x15, 0xf1, 0xec, - 0x8e, 0x54, 0xc8, 0x07, 0x51, 0x3f, 0xb2, 0x97, 0xd2, 0x78, 0x7a, 0x45, 0x0f, 0x00, 0x82, 0xd0, - 0xa7, 0x64, 0x3c, 0xb4, 0x8c, 0x40, 0x4d, 0x3f, 0x4c, 0x3f, 0x4a, 0xe3, 0xa2, 0x90, 0x74, 0x8c, - 0x00, 0x35, 0xa0, 0x14, 0x37, 0x67, 0xa0, 0x66, 0x6e, 0x5e, 0x62, 0x31, 0x4b, 0x0c, 0x04, 0x6a, - 0x58, 0x3a, 0x09, 0x69, 0xa0, 0x66, 0xb9, 0x8d, 0xfb, 0x57, 0x6c, 0x24, 0xd8, 0x09, 0x38, 0xda, - 0x83, 0x8d, 0x31, 0x39, 0x1f, 0xb2, 0xfd, 0x3b, 0x4c, 0x7e, 0x3c, 0x75, 0xb1, 0x41, 0xad, 0xb1, - 0xa8, 0x4f, 0xbc, 0x36, 0x26, 0xe7, 0xb3, 0x5b, 0xf9, 0x73, 0x28, 0xce, 0x2e, 0x71, 0xb5, 0xca, - 0xef, 0x5d, 0xad, 0xa9, 0x5b, 0x56, 0x6b, 0xb9, 0x05, 0x39, 0x31, 0x50, 0x17, 0xf2, 0x20, 0xdf, - 0x2a, 0x0f, 0xe5, 0x27, 0x90, 0xe5, 0x73, 0x16, 0x21, 0xc8, 0x04, 0xa7, 0x96, 0xc7, 0x43, 0x58, - 0xc7, 0xfc, 0x8c, 0xee, 0x40, 0xf6, 0x98, 0x86, 0xfa, 0x09, 0xf7, 0x6e, 0x1d, 0x8b, 0x4b, 0xf9, - 0x8f, 0x32, 0x40, 0x3c, 0x68, 0xd1, 0xa7, 0x50, 0x8a, 0x9f, 0xb9, 0xd1, 0xfb, 0x49, 0x3c, 0x52, - 0x20, 0x4d, 0x6c, 0x9b, 0xf7, 0x79, 0x01, 0xb3, 0x23, 0xba, 0x0b, 0x39, 0x7a, 0xee, 0x91, 0xa8, - 0x85, 0x0b, 0x38, 0xba, 0x31, 0x0f, 0x0d, 0xdf, 0x15, 0xfd, 0x59, 0xc0, 0xfc, 0x5c, 0xfe, 0xbb, - 0x0c, 0x6b, 0x8d, 0xf9, 0x21, 0x9d, 0x0b, 0xdc, 0x89, 0xaf, 0x8b, 0xdd, 0xbf, 0xb1, 0xc4, 0x11, - 0x01, 0x1f, 0x5c, 0x78, 0x14, 0x47, 0x50, 0x16, 0x82, 0x41, 0x83, 0xd0, 0x72, 0x92, 0x5d, 0x73, - 0x2d, 0x33, 0x89, 0x47, 0xcf, 0xa0, 0xa8, 0xbb, 0xb6, 0x6d, 0x31, 0x10, 0x0f, 0xe4, 0x5b, 0xc8, - 0x31, 0x9a, 0xf5, 0x7f, 0x86, 0xed, 0x00, 0xf4, 0x09, 0xe4, 0x03, 0xd7, 0x0f, 0x45, 0xcf, 0xcb, - 0x37, 0xeb, 0x86, 0x1c, 0x63, 0xd4, 0x2f, 0xd0, 0x47, 0x50, 0x24, 0x81, 0x4e, 0x1d, 0xc3, 0x72, - 0x4c, 0xee, 0x7c, 0x01, 0xc7, 0x02, 0xb4, 0x0d, 0x25, 0x67, 0x62, 0xdb, 0xc1, 0xf0, 0xd8, 0xf2, - 0x83, 0x30, 0x4a, 0x34, 0x70, 0xd1, 0x73, 0x26, 0x41, 0x6b, 0x20, 0x9f, 0xf2, 0x54, 0xa7, 0xb1, - 0x7c, 0x5a, 0xfe, 0x87, 0x0c, 0xf9, 0x68, 0x8f, 0xa1, 0x5d, 0xc8, 0x87, 0xc4, 0x37, 0x69, 0x38, - 0xfd, 0xac, 0x9b, 0x4b, 0x16, 0xde, 0x80, 0x23, 0xf0, 0x14, 0xb9, 0xd8, 0xdb, 0xa9, 0xef, 0xa0, - 0xb7, 0xd3, 0xb7, 0xab, 0xe9, 0x3e, 0x40, 0xec, 0x18, 0xff, 0xb8, 0x24, 0x24, 0x43, 0x77, 0xc4, - 0xac, 0xcf, 0x5a, 0xf4, 0xba, 0xff, 0x67, 0xc0, 0x10, 0x57, 0xaa, 0x87, 0xb3, 0x4d, 0xb9, 0x0e, - 0xa5, 0xc4, 0xea, 0x2d, 0xe7, 0x20, 0xc3, 0x16, 0xea, 0xce, 0x3f, 0x65, 0x50, 0x16, 0x17, 0x19, - 0xd2, 0x60, 0xab, 0xb6, 0xbf, 0x8f, 0x5b, 0xfb, 0xb5, 0x41, 0x6b, 0x88, 0x6b, 0xdd, 0xfd, 0xd6, - 0xb0, 0x77, 0x38, 0xec, 0x74, 0x8f, 0x6a, 0x2f, 0x3a, 0x4d, 0x45, 0x42, 0x1f, 0x81, 0xba, 0x44, - 0xdf, 0xe8, 0xbd, 0xea, 0x0e, 0x14, 0x19, 0x6d, 0xc1, 0xdd, 0x25, 0xda, 0xfe, 0xab, 0x97, 0x4a, - 0x6a, 0x85, 0xee, 0x65, 0xed, 0x33, 0x25, 0xbd, 0x4a, 0xd7, 0xe9, 0x2a, 0x99, 0x15, 0x2f, 0xd6, - 0x7f, 0x3d, 0x68, 0xf5, 0x95, 0xec, 0xce, 0x1f, 0xd2, 0xf0, 0xbd, 0x2b, 0x4b, 0x01, 0x6d, 0xc3, - 0xfd, 0x98, 0x73, 0xd4, 0x6a, 0x0c, 0x7a, 0x78, 0x3e, 0x8c, 0xfb, 0x70, 0x6f, 0x19, 0x80, 0x79, - 0x2a, 0xaf, 0x52, 0x32, 0x57, 0x53, 0x2b, 0x95, 0x9d, 0xae, 0x92, 0x46, 0x0f, 0x60, 0x73, 0x99, - 0x52, 0xa4, 0x27, 0xb3, 0x8a, 0x5b, 0x3b, 0xda, 0x57, 0xb2, 0xf3, 0x99, 0x4f, 0xb8, 0x34, 0x68, - 0x36, 0x5b, 0x47, 0x4a, 0xee, 0x1a, 0xfd, 0x51, 0x0d, 0x2b, 0xf9, 0x55, 0x31, 0xd7, 0x7b, 0x83, - 0x41, 0xef, 0xe5, 0x81, 0x52, 0x98, 0x4f, 0x64, 0x0c, 0x18, 0xf4, 0x0e, 0x0f, 0x94, 0xe2, 0x2a, - 0x6d, 0xbf, 0x87, 0x07, 0x0a, 0xa0, 0x8f, 0xe1, 0xc1, 0x2a, 0xed, 0xb0, 0xd9, 0xea, 0x37, 0x94, - 0xd2, 0xce, 0x67, 0x50, 0xec, 0xbb, 0x7e, 0xd8, 0xf3, 0x0d, 0xea, 0xa3, 0xbb, 0x80, 0xb8, 0xae, - 0x87, 0x9b, 0x2d, 0x9c, 0xc8, 0xbb, 0x0a, 0x77, 0x12, 0xf2, 0x5a, 0xbf, 0xd1, 0xea, 0x36, 0x3b, - 0xdd, 0x7d, 0x45, 0x46, 0x9b, 0xf0, 0x61, 0x42, 0xc3, 0x6c, 0x46, 0xaa, 0x54, 0x7d, 0xf2, 0xe6, - 0xad, 0x26, 0x7d, 0xf5, 0x56, 0x93, 0xbe, 0x79, 0xab, 0xc9, 0xbf, 0xbf, 0xd4, 0xe4, 0xbf, 0x5e, - 0x6a, 0xf2, 0x97, 0x97, 0x9a, 0xfc, 0xe6, 0x52, 0x93, 0xff, 0x73, 0xa9, 0xc9, 0xff, 0xbd, 0xd4, - 0xa4, 0x6f, 0x2e, 0x35, 0xf9, 0x8b, 0x77, 0x9a, 0xf4, 0xe6, 0x9d, 0x26, 0x7d, 0xf5, 0x4e, 0x93, - 0x7e, 0xf3, 0xcb, 0xf7, 0xfa, 0x65, 0x14, 0xff, 0x78, 0x1e, 0xe5, 0xb8, 0x64, 0xf7, 0xff, 0x01, - 0x00, 0x00, 0xff, 0xff, 0xd1, 0xf5, 0x32, 0xc4, 0x76, 0x0f, 0x00, 0x00, + // 1617 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0x3f, 0x73, 0xdb, 0xc8, + 0x15, 0x27, 0xf8, 0x4f, 0xe4, 0xa3, 0xa4, 0x43, 0x36, 0x3e, 0x1b, 0x92, 0xce, 0x90, 0x0f, 0x29, + 0x62, 0x2b, 0x17, 0x32, 0xb6, 0x26, 0xc9, 0xf8, 0x6e, 0xce, 0x19, 0x52, 0x84, 0x49, 0x46, 0x12, + 0xa9, 0x59, 0x42, 0xca, 0x39, 0x0d, 0xb2, 0x02, 0x56, 0x30, 0x2c, 0x10, 0xc0, 0x00, 0xa0, 0x47, + 0x4e, 0x73, 0xa9, 0x92, 0x49, 0x95, 0x2b, 0xf3, 0x01, 0x52, 0xa4, 0xcc, 0x17, 0xc8, 0x4c, 0x26, + 0xd5, 0x95, 0xee, 0x72, 0x93, 0xe2, 0x12, 0xcb, 0x4d, 0x8a, 0x14, 0xf7, 0x11, 0x32, 0xbb, 0x00, + 0x09, 0x90, 0x26, 0x15, 0xd9, 0x73, 0x8d, 0x66, 0xf7, 0xbd, 0xdf, 0xef, 0xed, 0x7b, 0x6f, 0xdf, + 0x7b, 0x0b, 0x11, 0x7e, 0xec, 0x9f, 0x5b, 0x0d, 0xea, 0x5a, 0xb6, 0x4b, 0x1b, 0xb6, 0x1b, 0xd1, + 0xc0, 0x25, 0x4e, 0xc3, 0x0f, 0xbc, 0xc8, 0x6b, 0xf8, 0x4f, 0x5f, 0x84, 0xb6, 0x41, 0x1c, 0xff, + 0x34, 0xb3, 0xac, 0x73, 0x1d, 0x5a, 0x73, 0xbc, 0x73, 0xbb, 0x3e, 0x11, 0x6f, 0xde, 0xb0, 0x3c, + 0xcb, 0x8b, 0x59, 0x6c, 0x15, 0x83, 0x36, 0x65, 0xcb, 0xf3, 0x2c, 0x87, 0xc6, 0xe6, 0x4e, 0xc7, + 0x67, 0x0d, 0x73, 0x1c, 0x90, 0xc8, 0xf6, 0xdc, 0x44, 0xbf, 0x3d, 0xaf, 0x8f, 0xec, 0x11, 0x0d, + 0x23, 0x32, 0xf2, 0x13, 0xc0, 0xc3, 0xe5, 0xce, 0xd1, 0x0b, 0x3f, 0xa0, 0x61, 0x68, 0x7b, 0xae, + 0x7f, 0x3a, 0xb3, 0x49, 0xa8, 0xf7, 0x96, 0x53, 0xc7, 0x8e, 0x6d, 0xf2, 0x3f, 0x31, 0x54, 0xf9, + 0x15, 0x14, 0x8f, 0x1c, 0xe2, 0xa2, 0x7b, 0x50, 0x72, 0x3d, 0x93, 0x86, 0x92, 0x70, 0xa7, 0x70, + 0xb7, 0xf6, 0xe0, 0xbb, 0xf5, 0x99, 0x18, 0xeb, 0x7d, 0xcf, 0xa4, 0x38, 0x46, 0xa0, 0x1f, 0x42, + 0x89, 0x9a, 0x16, 0x0d, 0xa5, 0x3c, 0x87, 0xde, 0x9a, 0x83, 0x32, 0x73, 0xaa, 0x69, 0x51, 0x1c, + 0xa3, 0x94, 0x00, 0x2a, 0x13, 0x11, 0xda, 0x85, 0xb2, 0x4f, 0x02, 0xea, 0x46, 0x92, 0x70, 0x47, + 0xb8, 0x5b, 0x7b, 0xf0, 0xfe, 0x82, 0x63, 0x7a, 0xed, 0x56, 0xf1, 0xcb, 0xaf, 0xb7, 0x73, 0x38, + 0x81, 0xa2, 0xfb, 0x50, 0x32, 0x9e, 0xda, 0x8e, 0x29, 0xe5, 0xff, 0x3f, 0x27, 0x46, 0x2a, 0x9f, + 0x43, 0x39, 0x16, 0xa3, 0x31, 0x94, 0x9e, 0x13, 0x67, 0x4c, 0x93, 0x03, 0x6f, 0xd5, 0x79, 0xec, + 0xdc, 0xc2, 0xf3, 0xfb, 0xf5, 0x23, 0x96, 0x83, 0xe3, 0x83, 0x5e, 0xbb, 0xf5, 0x98, 0xd1, 0xff, + 0xf9, 0xf5, 0xf6, 0x23, 0xcb, 0x8e, 0x9e, 0x8e, 0x4f, 0xeb, 0x86, 0x37, 0x6a, 0x58, 0x01, 0x39, + 0x23, 0x2e, 0x69, 0x30, 0x74, 0xe3, 0xf9, 0x6e, 0xe3, 0xea, 0xac, 0xd6, 0x99, 0x1d, 0x1c, 0x9f, + 0xa6, 0xfc, 0xa5, 0x04, 0x45, 0xe6, 0x01, 0xfa, 0x01, 0xe4, 0x6d, 0xf3, 0x3a, 0xd1, 0xe6, 0x6d, + 0x13, 0x75, 0xe1, 0x3d, 0x62, 0x59, 0x01, 0xb5, 0x48, 0x44, 0xf5, 0x80, 0xb8, 0x16, 0x4d, 0x62, + 0xbe, 0x3d, 0xc7, 0x6c, 0x4e, 0x50, 0x98, 0x81, 0xba, 0x39, 0xbc, 0x4e, 0x66, 0x24, 0x68, 0x1f, + 0xc4, 0xd4, 0xd2, 0x73, 0x6a, 0x44, 0x5e, 0x20, 0x15, 0xb8, 0x29, 0x79, 0x99, 0xa9, 0x13, 0x8e, + 0xea, 0xe6, 0x70, 0xea, 0x43, 0x2c, 0x42, 0x3f, 0x82, 0x62, 0x68, 0x10, 0x57, 0x2a, 0x72, 0x03, + 0x9b, 0x73, 0x06, 0xda, 0x24, 0x22, 0x83, 0xd3, 0x67, 0x43, 0x83, 0xb8, 0xdd, 0x1c, 0xe6, 0x48, + 0xd4, 0x80, 0xf2, 0x99, 0xed, 0x44, 0x34, 0x90, 0x4a, 0x0b, 0x23, 0x7f, 0xcc, 0x95, 0xdd, 0x1c, + 0x4e, 0x60, 0xe8, 0x23, 0x28, 0x39, 0xf6, 0xc8, 0x8e, 0xa4, 0x32, 0xc7, 0xdf, 0x98, 0xc3, 0x1f, + 0x30, 0x5d, 0x37, 0x87, 0x63, 0x10, 0xfa, 0x04, 0xc0, 0x0f, 0xbc, 0x67, 0xd4, 0x60, 0xfd, 0x24, + 0xad, 0x70, 0xca, 0xc6, 0x7c, 0x19, 0x4e, 0x01, 0xdd, 0x1c, 0xce, 0xc0, 0x51, 0x0b, 0xd6, 0x0c, + 0xcf, 0x19, 0x8f, 0x5c, 0xdd, 0xf0, 0x46, 0x3e, 0x89, 0xa4, 0x0a, 0xe7, 0x6f, 0xcd, 0xf1, 0xf7, + 0x38, 0x66, 0x8f, 0x43, 0xba, 0x39, 0xbc, 0x6a, 0x64, 0xf6, 0x68, 0x17, 0x2a, 0x2c, 0x4e, 0x3d, + 0xa4, 0x91, 0x54, 0xe5, 0xf4, 0x9b, 0x73, 0x74, 0x96, 0x8e, 0x21, 0x65, 0xcc, 0x95, 0x30, 0x5e, + 0xa2, 0x1d, 0x28, 0x45, 0x9e, 0xaf, 0x9f, 0x4b, 0xc0, 0x19, 0xf3, 0x2d, 0xa6, 0x79, 0xfe, 0x3e, + 0x4b, 0x60, 0xe4, 0xf9, 0xfb, 0xe8, 0x11, 0xd4, 0x7c, 0x12, 0x10, 0xc7, 0xa1, 0x8e, 0xfd, 0x6b, + 0x2a, 0xd5, 0x16, 0x66, 0xfe, 0x28, 0x45, 0x74, 0x73, 0x38, 0x4b, 0x40, 0xf7, 0xa0, 0xf8, 0xcc, + 0xb3, 0x5d, 0x69, 0x75, 0xe1, 0x51, 0x3f, 0xf7, 0x6c, 0x7e, 0x57, 0x0c, 0xd2, 0x2a, 0x43, 0xf1, + 0xdc, 0x76, 0x4d, 0xe5, 0xbf, 0x79, 0x58, 0x9f, 0xad, 0x2b, 0x16, 0xa6, 0x15, 0x78, 0x63, 0xdf, + 0x76, 0xad, 0x69, 0xff, 0xcc, 0x5a, 0xea, 0x24, 0x6a, 0x3c, 0x05, 0xa2, 0x4f, 0xa1, 0xea, 0xf9, + 0x34, 0x9e, 0x75, 0xbc, 0x7c, 0xd7, 0x1f, 0x6c, 0x5f, 0x59, 0xbe, 0x03, 0x1f, 0xa7, 0x0c, 0xf4, + 0x31, 0x94, 0xc2, 0x88, 0x04, 0x51, 0x52, 0xae, 0x9b, 0xf5, 0x78, 0x4e, 0xd6, 0x27, 0x73, 0xb2, + 0xae, 0x4d, 0xe6, 0x64, 0xab, 0xc2, 0x1a, 0xe7, 0x8b, 0x7f, 0x6d, 0x0b, 0x38, 0xa6, 0xa0, 0x9f, + 0x40, 0x81, 0xba, 0xe6, 0xb4, 0x4e, 0xaf, 0xc3, 0x64, 0x04, 0xf4, 0x53, 0x28, 0x86, 0x11, 0xf5, + 0x93, 0x62, 0xdd, 0x78, 0x83, 0xd8, 0x4e, 0x46, 0x77, 0xcc, 0xfb, 0x23, 0xe3, 0x71, 0x02, 0x7a, + 0x08, 0xa5, 0xb8, 0x4d, 0xcb, 0xd7, 0x67, 0xc6, 0x0c, 0xe5, 0x02, 0x2a, 0x93, 0xe4, 0xa1, 0x4f, + 0x60, 0x25, 0x2e, 0xaf, 0xc9, 0xf8, 0xfd, 0x30, 0x4e, 0x58, 0x3a, 0xda, 0x93, 0x72, 0x54, 0xa7, + 0x02, 0x3c, 0x61, 0xa0, 0x06, 0x14, 0x47, 0x9e, 0x49, 0x93, 0x54, 0x6f, 0x2d, 0xb9, 0xa0, 0x43, + 0x36, 0xc0, 0x39, 0x50, 0xf9, 0xad, 0x00, 0xef, 0xcd, 0x75, 0xfd, 0xbb, 0xdd, 0xf4, 0xa3, 0xec, + 0x4d, 0x17, 0xf8, 0xf1, 0x77, 0xae, 0x9e, 0x2e, 0x33, 0x57, 0xad, 0xfc, 0x29, 0x0f, 0xb5, 0xcc, + 0xf4, 0x40, 0x9b, 0x50, 0x71, 0x3c, 0x23, 0x36, 0xc7, 0x9c, 0xa8, 0xe2, 0xe9, 0x1e, 0x49, 0xb0, + 0x12, 0x26, 0xfd, 0xce, 0x02, 0x2d, 0xe0, 0xc9, 0x16, 0xdd, 0x06, 0x08, 0xa3, 0x80, 0x92, 0x91, + 0x6e, 0x9b, 0xa1, 0x54, 0xb8, 0x53, 0xb8, 0x5b, 0xc0, 0xd5, 0x58, 0xd2, 0x33, 0x43, 0xb4, 0x07, + 0xb5, 0xb4, 0xf9, 0x43, 0xa9, 0x78, 0xdd, 0xfc, 0x66, 0x59, 0xf1, 0xc0, 0xa1, 0xa6, 0x6d, 0x90, + 0x88, 0x86, 0x52, 0x89, 0xdb, 0xd8, 0x7a, 0xc3, 0x46, 0x86, 0x9d, 0x81, 0xa3, 0x47, 0xb0, 0x3e, + 0x22, 0x17, 0x3a, 0x7b, 0xdf, 0xf5, 0x6c, 0xb5, 0x48, 0xf3, 0x03, 0xc0, 0x1e, 0xc5, 0x0d, 0x81, + 0x57, 0x47, 0xe4, 0x62, 0xba, 0x53, 0x3e, 0x87, 0xea, 0x74, 0x93, 0xb6, 0x87, 0xf0, 0xce, 0xed, + 0x91, 0x7f, 0xcb, 0xf6, 0x50, 0x54, 0x28, 0xc7, 0x03, 0x7b, 0x2e, 0x0f, 0xc2, 0x5b, 0xe5, 0x41, + 0xb9, 0x0f, 0x25, 0x3e, 0xc7, 0x11, 0x82, 0x62, 0x78, 0x6e, 0xfb, 0x3c, 0x84, 0x35, 0xcc, 0xd7, + 0xe8, 0x06, 0x94, 0xce, 0x68, 0x64, 0x3c, 0xe5, 0xde, 0xad, 0xe1, 0x78, 0xa3, 0xfc, 0x5e, 0x00, + 0x48, 0x07, 0x39, 0xfa, 0x14, 0x6a, 0xe9, 0x31, 0xd7, 0x3a, 0x3f, 0x8b, 0x47, 0x22, 0x14, 0x88, + 0xe3, 0xf0, 0x4a, 0xad, 0x60, 0xb6, 0x44, 0x37, 0xa1, 0x4c, 0x2f, 0x7c, 0x92, 0xcc, 0x8c, 0x0a, + 0x4e, 0x76, 0xcc, 0x43, 0x33, 0xf0, 0xe2, 0x81, 0x50, 0xc1, 0x7c, 0xad, 0xfc, 0x55, 0x80, 0xd5, + 0xbd, 0xd9, 0x47, 0xa0, 0x1c, 0x7a, 0xe3, 0xc0, 0x88, 0xbf, 0x2d, 0xd6, 0x17, 0x38, 0x12, 0xc3, + 0xb5, 0x17, 0x3e, 0xc5, 0x09, 0x94, 0x85, 0x60, 0xd2, 0x30, 0xb2, 0xdd, 0xec, 0x7c, 0xbc, 0x92, + 0x99, 0xc5, 0xa3, 0x87, 0x50, 0x35, 0x3c, 0xc7, 0xb1, 0xc3, 0xb4, 0xe5, 0xae, 0x24, 0xa7, 0x68, + 0xe5, 0x0f, 0x02, 0x14, 0xd9, 0x1b, 0x83, 0x3e, 0x86, 0x95, 0xd0, 0x0b, 0x22, 0xfd, 0xf4, 0x45, + 0x52, 0x44, 0xd7, 0xe8, 0x86, 0x32, 0x63, 0xb4, 0x5e, 0xa0, 0x0f, 0xa0, 0x4a, 0x42, 0x83, 0xba, + 0x26, 0x1b, 0x14, 0x79, 0x9e, 0x9d, 0x54, 0x80, 0xb6, 0xa1, 0xe6, 0x8e, 0x1d, 0x27, 0xd4, 0xcf, + 0xec, 0x20, 0x8c, 0x92, 0x44, 0x03, 0x17, 0x3d, 0x66, 0x12, 0xb4, 0x0a, 0xc2, 0x39, 0x4f, 0x75, + 0x01, 0x0b, 0xe7, 0xca, 0xdf, 0x04, 0x58, 0x49, 0xde, 0x49, 0xb4, 0x0b, 0x2b, 0x11, 0x09, 0x2c, + 0x1a, 0x4d, 0xae, 0x75, 0x63, 0xc1, 0x83, 0xaa, 0x71, 0x04, 0x9e, 0x20, 0xe7, 0x7b, 0x3b, 0xff, + 0x2d, 0xf4, 0x76, 0xe1, 0xed, 0x6a, 0x7a, 0x08, 0x90, 0x3a, 0xc6, 0x2f, 0x97, 0x44, 0x44, 0xf7, + 0x4e, 0x99, 0xf5, 0x69, 0x8b, 0x5e, 0xf5, 0xbd, 0x04, 0x66, 0xbc, 0xa5, 0x46, 0x34, 0x7d, 0x89, + 0xd7, 0xa0, 0x96, 0x79, 0xda, 0x95, 0x32, 0x14, 0xd9, 0x83, 0xbd, 0xf3, 0x0f, 0x01, 0x56, 0xb3, + 0xe3, 0x1c, 0x6d, 0xc0, 0xfb, 0x1d, 0x3c, 0x38, 0x3e, 0xea, 0xf5, 0x3b, 0xfa, 0xe1, 0xa0, 0xad, + 0xea, 0xbd, 0xfe, 0x49, 0xf3, 0xa0, 0xd7, 0x16, 0x73, 0xe8, 0xfb, 0xf0, 0xbd, 0x59, 0xd5, 0x74, + 0xd7, 0x7a, 0xa2, 0xab, 0x87, 0x47, 0xda, 0x13, 0x7d, 0xa8, 0x6a, 0xa2, 0x70, 0x35, 0xf0, 0xa0, + 0xd9, 0x52, 0x0f, 0x38, 0x30, 0x8f, 0x3e, 0x82, 0xbb, 0x4b, 0x80, 0xbf, 0xe8, 0x69, 0xdd, 0xc1, + 0xb1, 0x96, 0x31, 0x5b, 0xb8, 0x06, 0x3a, 0xb5, 0x5d, 0xdc, 0xf9, 0xbb, 0x00, 0xe2, 0xfc, 0x37, + 0x01, 0x92, 0x61, 0xb3, 0xd9, 0xe9, 0x60, 0xb5, 0xd3, 0xd4, 0x54, 0x1d, 0x37, 0xfb, 0x1d, 0x55, + 0x1f, 0x1c, 0x65, 0x42, 0xfc, 0x00, 0xa4, 0x05, 0xfa, 0xbd, 0xc1, 0x71, 0x9f, 0xc5, 0xb5, 0x09, + 0x37, 0x17, 0x68, 0x87, 0xc7, 0x87, 0x62, 0x7e, 0x89, 0xee, 0xb0, 0xf9, 0x99, 0x58, 0x58, 0xa6, + 0xeb, 0xf5, 0xc5, 0xe2, 0x92, 0x13, 0x5b, 0x4f, 0x34, 0x75, 0x28, 0x96, 0x76, 0x7e, 0x57, 0x80, + 0xef, 0xbc, 0xf1, 0xdc, 0xa1, 0x6d, 0xd8, 0x4a, 0x39, 0x27, 0xea, 0x9e, 0x36, 0xc0, 0xb3, 0x61, + 0x6c, 0xc1, 0xad, 0x45, 0x00, 0xe6, 0xa9, 0xb0, 0x4c, 0xc9, 0x5c, 0xcd, 0x2f, 0x55, 0xf6, 0xfa, + 0x62, 0x01, 0xdd, 0x86, 0x8d, 0x45, 0xca, 0x38, 0x3d, 0xc5, 0x65, 0xdc, 0xe6, 0x49, 0x47, 0x2c, + 0xcd, 0x66, 0x3e, 0xe3, 0x92, 0xd6, 0x6e, 0xab, 0x27, 0x62, 0xf9, 0x0a, 0xfd, 0x49, 0x13, 0x8b, + 0x2b, 0xcb, 0x62, 0x6e, 0x0d, 0x34, 0x6d, 0x70, 0xb8, 0x2f, 0x56, 0x66, 0x13, 0x99, 0x02, 0xb4, + 0xc1, 0xd1, 0xbe, 0x58, 0x5d, 0xa6, 0x1d, 0x0e, 0xb0, 0x26, 0x02, 0xfa, 0x10, 0x6e, 0x2f, 0xd3, + 0xea, 0x6d, 0x75, 0xb8, 0x27, 0xd6, 0x76, 0x3e, 0x83, 0xea, 0xd0, 0x0b, 0xa2, 0x41, 0x60, 0xd2, + 0x00, 0xdd, 0x04, 0xc4, 0x75, 0x03, 0xdc, 0x56, 0x71, 0x26, 0xef, 0x12, 0xdc, 0xc8, 0xc8, 0x9b, + 0xc3, 0x3d, 0xb5, 0xdf, 0xee, 0xf5, 0x3b, 0xa2, 0xc0, 0xda, 0x2a, 0xa3, 0x61, 0x36, 0x13, 0x55, + 0xbe, 0x35, 0x7e, 0xf9, 0x4a, 0xce, 0x7d, 0xf5, 0x4a, 0xce, 0x7d, 0xf3, 0x4a, 0x16, 0x7e, 0x73, + 0x29, 0x0b, 0x7f, 0xbe, 0x94, 0x85, 0x2f, 0x2f, 0x65, 0xe1, 0xe5, 0xa5, 0x2c, 0xfc, 0xfb, 0x52, + 0x16, 0xfe, 0x73, 0x29, 0xe7, 0xbe, 0xb9, 0x94, 0x85, 0x2f, 0x5e, 0xcb, 0xb9, 0x97, 0xaf, 0xe5, + 0xdc, 0x57, 0xaf, 0xe5, 0xdc, 0x2f, 0x7f, 0xf6, 0x4e, 0xff, 0x53, 0xa6, 0x3f, 0x3b, 0x9c, 0x96, + 0xb9, 0x64, 0xf7, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x8f, 0x91, 0x55, 0xcd, 0xb0, 0x10, 0x00, + 0x00, +} + +func (x GroupingMode) String() string { + s, ok := GroupingMode_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) } - func (x AggregateRangeOp) String() string { s, ok := AggregateRangeOp_name[int32(x)] if ok { @@ -1852,14 +1952,9 @@ func (this *AggregateRange) Equal(that interface{}) bool { } else if this == nil { return false } - if len(this.PartitionBy) != len(that1.PartitionBy) { + if !this.Grouping.Equal(that1.Grouping) { return false } - for i := range this.PartitionBy { - if !this.PartitionBy[i].Equal(that1.PartitionBy[i]) { - return false - } - } if this.Operation != that1.Operation { return false } @@ -1877,14 +1972,14 @@ func (this *AggregateRange) Equal(that interface{}) bool { } return true } -func (this *AggregateVector) Equal(that interface{}) bool { +func (this *Grouping) Equal(that interface{}) bool { if that == nil { return this == nil } - that1, ok := that.(*AggregateVector) + that1, ok := that.(*Grouping) if !ok { - that2, ok := that.(AggregateVector) + that2, ok := that.(Grouping) if ok { that1 = &that2 } else { @@ -1896,14 +1991,41 @@ func (this *AggregateVector) Equal(that interface{}) bool { } else if this == nil { return false } - if len(this.GroupBy) != len(that1.GroupBy) { + if len(this.Columns) != len(that1.Columns) { return false } - for i := range this.GroupBy { - if !this.GroupBy[i].Equal(that1.GroupBy[i]) { + for i := range this.Columns { + if !this.Columns[i].Equal(that1.Columns[i]) { return false } } + if this.Mode != that1.Mode { + return false + } + return true +} +func (this *AggregateVector) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*AggregateVector) + if !ok { + that2, ok := that.(AggregateVector) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Grouping.Equal(that1.Grouping) { + return false + } if this.Operation != that1.Operation { return false } @@ -2431,8 +2553,8 @@ func (this *AggregateRange) GoString() string { } s := make([]string, 0, 10) s = append(s, "&physicalpb.AggregateRange{") - if this.PartitionBy != nil { - s = append(s, "PartitionBy: "+fmt.Sprintf("%#v", this.PartitionBy)+",\n") + if this.Grouping != nil { + s = append(s, "Grouping: "+fmt.Sprintf("%#v", this.Grouping)+",\n") } s = append(s, "Operation: "+fmt.Sprintf("%#v", this.Operation)+",\n") s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") @@ -2442,14 +2564,27 @@ func (this *AggregateRange) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *Grouping) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&physicalpb.Grouping{") + if this.Columns != nil { + s = append(s, "Columns: "+fmt.Sprintf("%#v", this.Columns)+",\n") + } + s = append(s, "Mode: "+fmt.Sprintf("%#v", this.Mode)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} func (this *AggregateVector) GoString() string { if this == nil { return "nil" } s := make([]string, 0, 6) s = append(s, "&physicalpb.AggregateVector{") - if this.GroupBy != nil { - s = append(s, "GroupBy: "+fmt.Sprintf("%#v", this.GroupBy)+",\n") + if this.Grouping != nil { + s = append(s, "Grouping: "+fmt.Sprintf("%#v", this.Grouping)+",\n") } s = append(s, "Operation: "+fmt.Sprintf("%#v", this.Operation)+",\n") s = append(s, "}") @@ -3062,10 +3197,50 @@ func (m *AggregateRange) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x10 } - if len(m.PartitionBy) > 0 { - for iNdEx := len(m.PartitionBy) - 1; iNdEx >= 0; iNdEx-- { + if m.Grouping != nil { + { + size, err := m.Grouping.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPhysicalpb(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Grouping) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Grouping) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Grouping) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Mode != 0 { + i = encodeVarintPhysicalpb(dAtA, i, uint64(m.Mode)) + i-- + dAtA[i] = 0x10 + } + if len(m.Columns) > 0 { + for iNdEx := len(m.Columns) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.PartitionBy[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Columns[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -3102,21 +3277,19 @@ func (m *AggregateVector) MarshalToSizedBuffer(dAtA []byte) (int, error) { if m.Operation != 0 { i = encodeVarintPhysicalpb(dAtA, i, uint64(m.Operation)) i-- - dAtA[i] = 0x10 + dAtA[i] = 0x18 } - if len(m.GroupBy) > 0 { - for iNdEx := len(m.GroupBy) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.GroupBy[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintPhysicalpb(dAtA, i, uint64(size)) + if m.Grouping != nil { + { + size, err := m.Grouping.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err } - i-- - dAtA[i] = 0xa + i -= size + i = encodeVarintPhysicalpb(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0xa } return len(dAtA) - i, nil } @@ -3182,21 +3355,21 @@ func (m *DataObjScan) MarshalToSizedBuffer(dAtA []byte) (int, error) { } } if len(m.StreamIds) > 0 { - dAtA22 := make([]byte, len(m.StreamIds)*10) - var j21 int + dAtA24 := make([]byte, len(m.StreamIds)*10) + var j23 int for _, num1 := range m.StreamIds { num := uint64(num1) for num >= 1<<7 { - dAtA22[j21] = uint8(uint64(num)&0x7f | 0x80) + dAtA24[j23] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j21++ + j23++ } - dAtA22[j21] = uint8(num) - j21++ + dAtA24[j23] = uint8(num) + j23++ } - i -= j21 - copy(dAtA[i:], dAtA22[:j21]) - i = encodeVarintPhysicalpb(dAtA, i, uint64(j21)) + i -= j23 + copy(dAtA[i:], dAtA24[:j23]) + i = encodeVarintPhysicalpb(dAtA, i, uint64(j23)) i-- dAtA[i] = 0x1a } @@ -3235,20 +3408,20 @@ func (m *TimeRange) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n23, err23 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err23 != nil { - return 0, err23 + n25, err25 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err25 != nil { + return 0, err25 } - i -= n23 - i = encodeVarintPhysicalpb(dAtA, i, uint64(n23)) + i -= n25 + i = encodeVarintPhysicalpb(dAtA, i, uint64(n25)) i-- dAtA[i] = 0x12 - n24, err24 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err24 != nil { - return 0, err24 + n26, err26 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err26 != nil { + return 0, err26 } - i -= n24 - i = encodeVarintPhysicalpb(dAtA, i, uint64(n24)) + i -= n26 + i = encodeVarintPhysicalpb(dAtA, i, uint64(n26)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -3860,11 +4033,9 @@ func (m *AggregateRange) Size() (n int) { } var l int _ = l - if len(m.PartitionBy) > 0 { - for _, e := range m.PartitionBy { - l = e.Size() - n += 1 + l + sovPhysicalpb(uint64(l)) - } + if m.Grouping != nil { + l = m.Grouping.Size() + n += 1 + l + sovPhysicalpb(uint64(l)) } if m.Operation != 0 { n += 1 + sovPhysicalpb(uint64(m.Operation)) @@ -3880,18 +4051,34 @@ func (m *AggregateRange) Size() (n int) { return n } -func (m *AggregateVector) Size() (n int) { +func (m *Grouping) Size() (n int) { if m == nil { return 0 } var l int _ = l - if len(m.GroupBy) > 0 { - for _, e := range m.GroupBy { + if len(m.Columns) > 0 { + for _, e := range m.Columns { l = e.Size() n += 1 + l + sovPhysicalpb(uint64(l)) } } + if m.Mode != 0 { + n += 1 + sovPhysicalpb(uint64(m.Mode)) + } + return n +} + +func (m *AggregateVector) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Grouping != nil { + l = m.Grouping.Size() + n += 1 + l + sovPhysicalpb(uint64(l)) + } if m.Operation != 0 { n += 1 + sovPhysicalpb(uint64(m.Operation)) } @@ -4286,13 +4473,8 @@ func (this *AggregateRange) String() string { if this == nil { return "nil" } - repeatedStringForPartitionBy := "[]*ColumnExpression{" - for _, f := range this.PartitionBy { - repeatedStringForPartitionBy += strings.Replace(fmt.Sprintf("%v", f), "ColumnExpression", "expressionpb.ColumnExpression", 1) + "," - } - repeatedStringForPartitionBy += "}" s := strings.Join([]string{`&AggregateRange{`, - `PartitionBy:` + repeatedStringForPartitionBy + `,`, + `Grouping:` + strings.Replace(this.Grouping.String(), "Grouping", "Grouping", 1) + `,`, `Operation:` + fmt.Sprintf("%v", this.Operation) + `,`, `Start:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Start), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, `End:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.End), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, @@ -4302,17 +4484,28 @@ func (this *AggregateRange) String() string { }, "") return s } -func (this *AggregateVector) String() string { +func (this *Grouping) String() string { if this == nil { return "nil" } - repeatedStringForGroupBy := "[]*ColumnExpression{" - for _, f := range this.GroupBy { - repeatedStringForGroupBy += strings.Replace(fmt.Sprintf("%v", f), "ColumnExpression", "expressionpb.ColumnExpression", 1) + "," + repeatedStringForColumns := "[]*ColumnExpression{" + for _, f := range this.Columns { + repeatedStringForColumns += strings.Replace(fmt.Sprintf("%v", f), "ColumnExpression", "expressionpb.ColumnExpression", 1) + "," + } + repeatedStringForColumns += "}" + s := strings.Join([]string{`&Grouping{`, + `Columns:` + repeatedStringForColumns + `,`, + `Mode:` + fmt.Sprintf("%v", this.Mode) + `,`, + `}`, + }, "") + return s +} +func (this *AggregateVector) String() string { + if this == nil { + return "nil" } - repeatedStringForGroupBy += "}" s := strings.Join([]string{`&AggregateVector{`, - `GroupBy:` + repeatedStringForGroupBy + `,`, + `Grouping:` + strings.Replace(this.Grouping.String(), "Grouping", "Grouping", 1) + `,`, `Operation:` + fmt.Sprintf("%v", this.Operation) + `,`, `}`, }, "") @@ -5324,7 +5517,7 @@ func (m *AggregateRange) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PartitionBy", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Grouping", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5351,8 +5544,10 @@ func (m *AggregateRange) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.PartitionBy = append(m.PartitionBy, &expressionpb.ColumnExpression{}) - if err := m.PartitionBy[len(m.PartitionBy)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.Grouping == nil { + m.Grouping = &Grouping{} + } + if err := m.Grouping.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -5531,6 +5726,112 @@ func (m *AggregateRange) Unmarshal(dAtA []byte) error { } return nil } +func (m *Grouping) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPhysicalpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Grouping: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Grouping: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Columns", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPhysicalpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPhysicalpb + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPhysicalpb + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Columns = append(m.Columns, &expressionpb.ColumnExpression{}) + if err := m.Columns[len(m.Columns)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Mode", wireType) + } + m.Mode = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPhysicalpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Mode |= GroupingMode(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipPhysicalpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPhysicalpb + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthPhysicalpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *AggregateVector) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -5562,7 +5863,7 @@ func (m *AggregateVector) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field GroupBy", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Grouping", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5589,12 +5890,14 @@ func (m *AggregateVector) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.GroupBy = append(m.GroupBy, &expressionpb.ColumnExpression{}) - if err := m.GroupBy[len(m.GroupBy)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.Grouping == nil { + m.Grouping = &Grouping{} + } + if err := m.Grouping.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + case 3: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Operation", wireType) } diff --git a/pkg/engine/internal/proto/physicalpb/physicalpb.proto b/pkg/engine/internal/proto/physicalpb/physicalpb.proto index 5e0ba407e3d4a..bee0700ee20ff 100644 --- a/pkg/engine/internal/proto/physicalpb/physicalpb.proto +++ b/pkg/engine/internal/proto/physicalpb/physicalpb.proto @@ -53,8 +53,8 @@ message Node { // AggregateRange aggregates samples into windowed ranges. message AggregateRange { - // Columns to partition the data by. - repeated loki.expression.ColumnExpression partition_by = 1; + // Grouping for the data. + Grouping grouping = 1; AggregateRangeOp operation = 2; @@ -76,6 +76,23 @@ message AggregateRange { ]; } +// Grouping represents the grouping by/without label(s) for vector aggregators +// and range vector aggregators. +message Grouping { + repeated loki.expression.ColumnExpression columns = 1; + GroupingMode mode = 2; +} + +// GroupingMode represents the way on interpretation of the list of columns for grouping. +enum GroupingMode { + GROUPING_MODE_INVALID = 0; + + GROUPING_MODE_GROUPING_BY_EMPTY_SET = 1; + GROUPING_MODE_GROUPING_BY_LABEL_SET = 2; + GROUPING_MODE_GROUPING_WITHOUT_EMPTY_SET = 3; + GROUPING_MODE_GROUPING_WITHOUT_LABEL_SET = 4; +} + // AggregateRangeOp represents the operation to perform on the aggregated // data. enum AggregateRangeOp { @@ -91,11 +108,11 @@ enum AggregateRangeOp { // AggregateVector represents an operation to aggregate a range vector into an // instant vector, with optional grouping on specified dimensions. message AggregateVector { - // Columns to group the data by. If empty, all rows are aggregated into a single result. - repeated loki.expression.ColumnExpression group_by = 1; + // Grouping for the data. + Grouping grouping = 1; // Aggregation operation to perform over the underlying range vector. - AggregateVectorOp operation = 2; + AggregateVectorOp operation = 3; } // AggregateVectorOp represents the different aggregation operations that can diff --git a/pkg/engine/internal/proto/physicalpb/physicalpb_test.go b/pkg/engine/internal/proto/physicalpb/physicalpb_test.go index a0c816e3d8ad3..3efbe3efb48f7 100644 --- a/pkg/engine/internal/proto/physicalpb/physicalpb_test.go +++ b/pkg/engine/internal/proto/physicalpb/physicalpb_test.go @@ -112,8 +112,11 @@ func Test_Node(t *testing.T) { node: &physical.RangeAggregation{ NodeID: ulid.Make(), - PartitionBy: []physical.ColumnExpression{ - &physical.ColumnExpr{Ref: types.ColumnRef{Column: "partition_col", Type: types.ColumnTypeLabel}}, + Grouping: physical.Grouping{ + Columns: []physical.ColumnExpression{ + &physical.ColumnExpr{Ref: types.ColumnRef{Column: "partition_col", Type: types.ColumnTypeLabel}}, + }, + Mode: types.GroupingModeByLabelSet, }, Operation: types.RangeAggregationTypeCount, Start: time.Date(2024, 1, 1, 0, 0, 0, 0, time.UTC), @@ -127,8 +130,11 @@ func Test_Node(t *testing.T) { node: &physical.VectorAggregation{ NodeID: ulid.Make(), - GroupBy: []physical.ColumnExpression{ - &physical.ColumnExpr{Ref: types.ColumnRef{Column: "group_col", Type: types.ColumnTypeLabel}}, + Grouping: physical.Grouping{ + Columns: []physical.ColumnExpression{ + &physical.ColumnExpr{Ref: types.ColumnRef{Column: "group_col", Type: types.ColumnTypeLabel}}, + }, + Mode: types.GroupingModeByLabelSet, }, Operation: types.VectorAggregationTypeSum, }, diff --git a/pkg/engine/internal/proto/physicalpb/unmarshal_node.go b/pkg/engine/internal/proto/physicalpb/unmarshal_node.go index 9bd380fdd033f..b217799539860 100644 --- a/pkg/engine/internal/proto/physicalpb/unmarshal_node.go +++ b/pkg/engine/internal/proto/physicalpb/unmarshal_node.go @@ -137,7 +137,7 @@ func (n *AggregateRange) UnmarshalPhysical(from physical.Node) error { return fmt.Errorf("unsupported physical node type: %T", from) } - partitionBy, err := unmarshalColumnExpressions(rangeAgg.PartitionBy) + grouping, err := unmarshalGrouping(rangeAgg.Grouping) if err != nil { return err } @@ -148,17 +148,34 @@ func (n *AggregateRange) UnmarshalPhysical(from physical.Node) error { } *n = AggregateRange{ - PartitionBy: partitionBy, - Operation: op, - Start: rangeAgg.Start, - End: rangeAgg.End, - Step: rangeAgg.Step, - Range: rangeAgg.Range, + Grouping: grouping, + Operation: op, + Start: rangeAgg.Start, + End: rangeAgg.End, + Step: rangeAgg.Step, + Range: rangeAgg.Range, } return nil } +func unmarshalGrouping(g physical.Grouping) (*Grouping, error) { + columns, err := unmarshalColumnExpressions(g.Columns) + if err != nil { + return nil, err + } + + var mode GroupingMode + if err := mode.unmarshalType(g.Mode); err != nil { + return nil, err + } + + return &Grouping{ + Columns: columns, + Mode: mode, + }, nil +} + func unmarshalColumnExpressions(from []physical.ColumnExpression) ([]*expressionpb.ColumnExpression, error) { if from == nil { return nil, nil @@ -182,7 +199,7 @@ func (n *AggregateVector) UnmarshalPhysical(from physical.Node) error { return fmt.Errorf("unsupported physical node type: %T", from) } - groupBy, err := unmarshalColumnExpressions(vectorAgg.GroupBy) + grouping, err := unmarshalGrouping(vectorAgg.Grouping) if err != nil { return err } @@ -193,7 +210,7 @@ func (n *AggregateVector) UnmarshalPhysical(from physical.Node) error { } *n = AggregateVector{ - GroupBy: groupBy, + Grouping: grouping, Operation: op, } return nil diff --git a/pkg/engine/internal/proto/physicalpb/unmarshal_types.go b/pkg/engine/internal/proto/physicalpb/unmarshal_types.go index 56048a2879703..9b03984c2e5b2 100644 --- a/pkg/engine/internal/proto/physicalpb/unmarshal_types.go +++ b/pkg/engine/internal/proto/physicalpb/unmarshal_types.go @@ -30,6 +30,13 @@ var ( types.VectorAggregationTypeSort: AGGREGATE_VECTOR_OP_SORT, types.VectorAggregationTypeSortDesc: AGGREGATE_VECTOR_OP_SORT_DESC, } + + protoGroupingModeLookup = map[types.GroupingMode]GroupingMode{ + types.GroupingModeByEmptySet: GROUPING_MODE_GROUPING_BY_EMPTY_SET, + types.GroupingModeByLabelSet: GROUPING_MODE_GROUPING_BY_LABEL_SET, + types.GroupingModeWithoutEmptySet: GROUPING_MODE_GROUPING_WITHOUT_EMPTY_SET, + types.GroupingModeWithoutLabelSet: GROUPING_MODE_GROUPING_WITHOUT_LABEL_SET, + } ) func (op *AggregateRangeOp) unmarshalType(from types.RangeAggregationType) error { @@ -47,3 +54,11 @@ func (op *AggregateVectorOp) unmarshalType(from types.VectorAggregationType) err } return fmt.Errorf("unknown VectorAggregationType: %v", from) } + +func (m *GroupingMode) unmarshalType(from types.GroupingMode) error { + if result, ok := protoGroupingModeLookup[from]; ok { + *m = result + return nil + } + return fmt.Errorf("unknown GroupingMode: %v", from) +} diff --git a/pkg/engine/internal/types/aggregations.go b/pkg/engine/internal/types/aggregations.go index 6049b6b6cee56..4cff628e60d64 100644 --- a/pkg/engine/internal/types/aggregations.go +++ b/pkg/engine/internal/types/aggregations.go @@ -10,6 +10,7 @@ const ( RangeAggregationTypeSum // Represents sum_over_time range aggregation RangeAggregationTypeMax // Represents max_over_time range aggregation RangeAggregationTypeMin // Represents min_over_time range aggregation + RangeAggregationTypeAvg // Represents avg_over_time range aggregation RangeAggregationTypeBytes // Represents bytes_over_time range aggregation ) @@ -23,6 +24,8 @@ func (op RangeAggregationType) String() string { return "max" case RangeAggregationTypeMin: return "min" + case RangeAggregationTypeAvg: + return "avg" case RangeAggregationTypeBytes: return "bytes" default: @@ -59,6 +62,20 @@ func (op VectorAggregationType) String() string { return "min" case VectorAggregationTypeCount: return "count" + case VectorAggregationTypeAvg: + return "avg" + case VectorAggregationTypeStddev: + return "stddev" + case VectorAggregationTypeStdvar: + return "stdvar" + case VectorAggregationTypeBottomK: + return "bottomk" + case VectorAggregationTypeTopK: + return "topk" + case VectorAggregationTypeSort: + return "sort" + case VectorAggregationTypeSortDesc: + return "sort_desc" default: return "invalid" } diff --git a/pkg/engine/internal/types/grouping.go b/pkg/engine/internal/types/grouping.go new file mode 100644 index 0000000000000..4e16128c2255e --- /dev/null +++ b/pkg/engine/internal/types/grouping.go @@ -0,0 +1,30 @@ +package types + +import "fmt" + +// GroupingMode represents the grouping by/without label(s) for vector aggregators and range vector aggregators. +type GroupingMode int + +const ( + GroupingModeInvalid GroupingMode = iota + GroupingModeByEmptySet // Grouping by empty label set: by () () + GroupingModeByLabelSet // Grouping by label set: by () () + GroupingModeWithoutEmptySet // Grouping without empty label set: without () () + GroupingModeWithoutLabelSet // Grouping without label set: without () () +) + +// String returns the string representation of the GroupingMode. +func (t GroupingMode) String() string { + switch t { + case GroupingModeByEmptySet: + return "GROUPING_BY_EMPTY_SET" + case GroupingModeByLabelSet: + return "GROUPING_BY_LABEL_SET" + case GroupingModeWithoutEmptySet: + return "GROUPING_WITHOUT_EMPTY_SET" + case GroupingModeWithoutLabelSet: + return "GROUPING_WITHOUT_LABEL_SET" + default: + panic(fmt.Sprintf("unknown grouping mode %d", t)) + } +} diff --git a/pkg/logql/bench/bench_test.go b/pkg/logql/bench/bench_test.go index 644eed61f5278..607657d20acfe 100644 --- a/pkg/logql/bench/bench_test.go +++ b/pkg/logql/bench/bench_test.go @@ -110,7 +110,6 @@ func TestStorageEquality(t *testing.T) { TestCaseGeneratorConfig{RangeType: *rangeType, RangeInterval: *rangeInterval}, config, ).Generate() - return &store{ Name: name, Cases: cases, @@ -451,7 +450,7 @@ func assertVectorEqualWithTolerance(t *testing.T, expected, actual promql.Vector // assertMatrixEqualWithTolerance compares two Matrix instances with floating point tolerance func assertMatrixEqualWithTolerance(t *testing.T, expected, actual promql.Matrix, tolerance float64) { - require.Len(t, actual, len(expected)) + require.Equal(t, len(expected), len(actual), "number of entries differs") for i := range expected { e := expected[i] diff --git a/pkg/logql/bench/generator_query.go b/pkg/logql/bench/generator_query.go index e67990dba13ad..b968889ac13c8 100644 --- a/pkg/logql/bench/generator_query.go +++ b/pkg/logql/bench/generator_query.go @@ -213,6 +213,10 @@ func (g *TestCaseGenerator) Generate() []TestCase { fmt.Sprintf(`count_over_time(%s[%s])`, selector, rangeInterval), fmt.Sprintf(`count_over_time(%s | detected_level=~"error|warn" [%s])`, selector, rangeInterval), fmt.Sprintf(`count_over_time(%s |= "level" [%s])`, selector, rangeInterval), + //fmt.Sprintf(`avg_over_time(%s | json | unwrap rows_affected [%s])`, selector, rangeInterval), + //fmt.Sprintf(`avg_over_time(%s |= "level" | json | unwrap rows_affected [%s])`, selector, rangeInterval), + //fmt.Sprintf(`min_over_time(%s |= "level" | json | unwrap rows_affected [%s])`, selector, rangeInterval), + //fmt.Sprintf(`max_over_time(%s |= "level" | json | unwrap rows_affected [%s])`, selector, rangeInterval), fmt.Sprintf(`rate(%s | detected_level=~"error|warn" [%s])`, selector, rangeInterval), } @@ -246,6 +250,11 @@ func (g *TestCaseGenerator) Generate() []TestCase { addMetricQuery(fmt.Sprintf(`sum by (level) (sum_over_time({service_name="database"} | json | unwrap rows_affected [%s]))`, rangeInterval), start, end, step) addMetricQuery(fmt.Sprintf(`sum by (level) (sum_over_time({service_name="loki"} | logfmt | duration != "" | unwrap duration_seconds(duration) [%s]))`, rangeInterval), start, end, step) + //addMetricQuery(fmt.Sprintf(`max by (level) (min_over_time({service_name="loki"} | logfmt | duration != "" | unwrap duration_seconds(duration) [%s]))`, rangeInterval), start, end, step) + //addMetricQuery(fmt.Sprintf(`avg by (level) (avg_over_time({service_name="loki"} | logfmt | duration != "" | unwrap duration_seconds(duration) [%s]))`, rangeInterval), start, end, step) + //addMetricQuery(fmt.Sprintf(`max by () (avg_over_time({service_name="loki"} | logfmt | duration != "" | unwrap duration_seconds(duration) [%s]))`, rangeInterval), start, end, step) + //addMetricQuery(fmt.Sprintf(`max by (level) (avg_over_time({service_name="loki"} | logfmt | duration != "" | unwrap duration_seconds(duration) [%s]) without (service_name))`, rangeInterval), start, end, step) + //addMetricQuery(fmt.Sprintf(`max without () (sum_over_time({service_name="loki"} | logfmt | duration != "" | unwrap duration_seconds(duration) [%s]))`, rangeInterval), start, end, step) // Dense period queries for _, interval := range g.logGenCfg.DenseIntervals {