mirror of https://github.com/grafana/loki
You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
392 lines
12 KiB
392 lines
12 KiB
|
7 months ago
|
package executor
|
||
|
|
|
||
|
|
import (
|
||
|
6 months ago
|
"context"
|
||
|
7 months ago
|
"errors"
|
||
|
|
"fmt"
|
||
|
4 months ago
|
"slices"
|
||
|
|
"sort"
|
||
|
7 months ago
|
"time"
|
||
|
|
|
||
|
|
"github.com/apache/arrow-go/v18/arrow"
|
||
|
|
"github.com/apache/arrow-go/v18/arrow/array"
|
||
|
|
|
||
|
|
"github.com/grafana/loki/v3/pkg/engine/internal/datatype"
|
||
|
|
"github.com/grafana/loki/v3/pkg/engine/internal/types"
|
||
|
3 months ago
|
"github.com/grafana/loki/v3/pkg/engine/internal/planner/physical"
|
||
|
7 months ago
|
)
|
||
|
|
|
||
|
|
type rangeAggregationOptions struct {
|
||
|
|
partitionBy []physical.ColumnExpression
|
||
|
|
|
||
|
|
// start and end timestamps are equal for instant queries.
|
||
|
6 months ago
|
startTs time.Time // start timestamp of the query
|
||
|
|
endTs time.Time // end timestamp of the query
|
||
|
|
rangeInterval time.Duration // range interval
|
||
|
|
step time.Duration // step used for range queries
|
||
|
3 months ago
|
operation types.RangeAggregationType
|
||
|
7 months ago
|
}
|
||
|
|
|
||
|
3 months ago
|
var (
|
||
|
|
// rangeAggregationOperations holds the mapping of range aggregation types to operations for an aggregator.
|
||
|
|
rangeAggregationOperations = map[types.RangeAggregationType]aggregationOperation{
|
||
|
|
types.RangeAggregationTypeSum: aggregationOperationSum,
|
||
|
|
types.RangeAggregationTypeCount: aggregationOperationCount,
|
||
|
|
types.RangeAggregationTypeMax: aggregationOperationMax,
|
||
|
|
types.RangeAggregationTypeMin: aggregationOperationMin,
|
||
|
|
}
|
||
|
|
)
|
||
|
|
|
||
|
4 months ago
|
// window is a time interval where start is exclusive and end is inclusive
|
||
|
|
// Refer to [logql.batchRangeVectorIterator].
|
||
|
|
type window struct {
|
||
|
|
start, end time.Time
|
||
|
|
}
|
||
|
|
|
||
|
|
// Contains returns if the timestamp t is within the bounds of the window.
|
||
|
|
// The window start is exclusive, the window end is inclusive.
|
||
|
|
func (w window) Contains(t time.Time) bool {
|
||
|
|
return t.After(w.start) && !t.After(w.end)
|
||
|
|
}
|
||
|
|
|
||
|
|
// timestampMatchingWindowsFunc resolves matching range interval windows for a specific timestamp.
|
||
|
|
// The list can be empty if the timestamp is out of bounds or does not match any of the range windows.
|
||
|
|
type timestampMatchingWindowsFunc func(time.Time) []window
|
||
|
|
|
||
|
4 months ago
|
// rangeAggregationPipeline is a pipeline that performs aggregations over a time window.
|
||
|
6 months ago
|
//
|
||
|
|
// 1. It reads from the input pipelines
|
||
|
|
// 2. Partitions the data by the specified columns
|
||
|
|
// 3. Applies the aggregation function on each partition
|
||
|
|
//
|
||
|
7 months ago
|
// Current version only supports counting for instant queries.
|
||
|
4 months ago
|
type rangeAggregationPipeline struct {
|
||
|
|
inputs []Pipeline
|
||
|
|
inputsExhausted bool // indicates if all inputs are exhausted
|
||
|
|
|
||
|
|
aggregator *aggregator
|
||
|
4 months ago
|
windowsForTimestamp timestampMatchingWindowsFunc // function to find matching time windows for a given timestamp
|
||
|
|
evaluator expressionEvaluator // used to evaluate column expressions
|
||
|
4 months ago
|
opts rangeAggregationOptions
|
||
|
|
}
|
||
|
7 months ago
|
|
||
|
4 months ago
|
func newRangeAggregationPipeline(inputs []Pipeline, evaluator expressionEvaluator, opts rangeAggregationOptions) (*rangeAggregationPipeline, error) {
|
||
|
|
r := &rangeAggregationPipeline{
|
||
|
|
inputs: inputs,
|
||
|
|
evaluator: evaluator,
|
||
|
|
opts: opts,
|
||
|
|
}
|
||
|
|
r.init()
|
||
|
|
return r, nil
|
||
|
7 months ago
|
}
|
||
|
|
|
||
|
4 months ago
|
func (r *rangeAggregationPipeline) init() {
|
||
|
4 months ago
|
windows := []window{}
|
||
|
4 months ago
|
cur := r.opts.startTs
|
||
|
|
for cur.Compare(r.opts.endTs) <= 0 {
|
||
|
4 months ago
|
windows = append(windows, window{start: cur.Add(-r.opts.rangeInterval), end: cur})
|
||
|
4 months ago
|
|
||
|
|
if r.opts.step == 0 {
|
||
|
|
break
|
||
|
|
}
|
||
|
|
|
||
|
|
// advance to the next window using step
|
||
|
|
cur = cur.Add(r.opts.step)
|
||
|
|
}
|
||
|
|
|
||
|
4 months ago
|
f := newMatcherFactoryFromOpts(r.opts)
|
||
|
|
r.windowsForTimestamp = f.createMatcher(windows)
|
||
|
3 months ago
|
|
||
|
|
op, ok := rangeAggregationOperations[r.opts.operation]
|
||
|
|
if !ok {
|
||
|
|
panic(fmt.Sprintf("unknown range aggregation operation: %v", r.opts.operation))
|
||
|
|
}
|
||
|
|
|
||
|
|
r.aggregator = newAggregator(r.opts.partitionBy, len(windows), op)
|
||
|
7 months ago
|
}
|
||
|
|
|
||
|
|
// Read reads the next value into its state.
|
||
|
|
// It returns an error if reading fails or when the pipeline is exhausted. In this case, the function returns EOF.
|
||
|
|
// The implementation must retain the returned error in its state and return it with subsequent Value() calls.
|
||
|
4 months ago
|
func (r *rangeAggregationPipeline) Read(ctx context.Context) (arrow.Record, error) {
|
||
|
4 months ago
|
if r.inputsExhausted {
|
||
|
4 months ago
|
return nil, EOF
|
||
|
4 months ago
|
}
|
||
|
|
|
||
|
4 months ago
|
return r.read(ctx)
|
||
|
7 months ago
|
}
|
||
|
|
|
||
|
|
// 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.
|
||
|
3 months ago
|
// - Add toggle to return partial results on Read() call instead of returning only after exhausting all inputs.
|
||
|
4 months ago
|
func (r *rangeAggregationPipeline) read(ctx context.Context) (arrow.Record, error) {
|
||
|
7 months ago
|
var (
|
||
|
|
tsColumnExpr = &physical.ColumnExpr{
|
||
|
|
Ref: types.ColumnRef{
|
||
|
|
Column: types.ColumnNameBuiltinTimestamp,
|
||
|
|
Type: types.ColumnTypeBuiltin,
|
||
|
|
},
|
||
|
|
} // timestamp column expression
|
||
|
|
|
||
|
3 months ago
|
valColumnExpr = &physical.ColumnExpr{
|
||
|
|
Ref: types.ColumnRef{
|
||
|
|
Column: types.ColumnNameGeneratedValue,
|
||
|
|
Type: types.ColumnTypeGenerated,
|
||
|
|
},
|
||
|
|
} // value column expression
|
||
|
|
|
||
|
7 months ago
|
// reused on each row read
|
||
|
|
labelValues = make([]string, len(r.opts.partitionBy))
|
||
|
|
)
|
||
|
|
|
||
|
|
r.aggregator.Reset() // reset before reading new inputs
|
||
|
|
inputsExhausted := false
|
||
|
|
for !inputsExhausted {
|
||
|
|
inputsExhausted = true
|
||
|
|
|
||
|
|
for _, input := range r.inputs {
|
||
|
4 months ago
|
record, err := input.Read(ctx)
|
||
|
|
if err != nil {
|
||
|
7 months ago
|
if errors.Is(err, EOF) {
|
||
|
|
continue
|
||
|
|
}
|
||
|
|
return nil, err
|
||
|
|
}
|
||
|
|
|
||
|
|
inputsExhausted = false
|
||
|
4 months ago
|
defer record.Release()
|
||
|
7 months ago
|
|
||
|
|
// 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
|
||
|
|
}
|
||
|
|
|
||
|
6 months ago
|
if vec.Type() != datatype.Loki.String {
|
||
|
7 months ago
|
return nil, fmt.Errorf("unsupported datatype for partitioning %s", vec.Type())
|
||
|
|
}
|
||
|
|
|
||
|
|
arrays = append(arrays, vec.ToArray().(*array.String))
|
||
|
|
}
|
||
|
|
|
||
|
|
// extract timestamp column to check if the entry is in range
|
||
|
3 months ago
|
tsVec, err := r.evaluator.eval(tsColumnExpr, record)
|
||
|
7 months ago
|
if err != nil {
|
||
|
|
return nil, err
|
||
|
|
}
|
||
|
3 months ago
|
tsCol := tsVec.ToArray().(*array.Timestamp)
|
||
|
|
|
||
|
|
// no need to extract value column for COUNT aggregation
|
||
|
|
var valVec ColumnVector
|
||
|
|
if r.opts.operation != types.RangeAggregationTypeCount {
|
||
|
|
valVec, err = r.evaluator.eval(valColumnExpr, record)
|
||
|
|
if err != nil {
|
||
|
|
return nil, err
|
||
|
|
}
|
||
|
|
}
|
||
|
7 months ago
|
|
||
|
|
for row := range int(record.NumRows()) {
|
||
|
4 months ago
|
windows := r.windowsForTimestamp(tsCol.Value(row).ToTime(arrow.Nanosecond))
|
||
|
4 months ago
|
if len(windows) == 0 {
|
||
|
|
continue // out of range, skip this row
|
||
|
7 months ago
|
}
|
||
|
|
|
||
|
|
// reset label values and hash for each row
|
||
|
|
clear(labelValues)
|
||
|
|
for col, arr := range arrays {
|
||
|
|
labelValues[col] = arr.Value(row)
|
||
|
|
}
|
||
|
|
|
||
|
3 months ago
|
var value float64
|
||
|
|
if r.opts.operation != types.RangeAggregationTypeCount {
|
||
|
|
switch v := valVec.Value(row).(type) {
|
||
|
|
case float64:
|
||
|
|
value = v
|
||
|
|
case int64:
|
||
|
|
value = float64(v)
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
4 months ago
|
for _, w := range windows {
|
||
|
3 months ago
|
r.aggregator.Add(w.end, value, labelValues)
|
||
|
4 months ago
|
}
|
||
|
7 months ago
|
}
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
4 months ago
|
r.inputsExhausted = true
|
||
|
|
return r.aggregator.BuildRecord()
|
||
|
7 months ago
|
}
|
||
|
|
|
||
|
|
// Close closes the resources of the pipeline.
|
||
|
|
// The implementation must close all the of the pipeline's inputs.
|
||
|
4 months ago
|
func (r *rangeAggregationPipeline) Close() {
|
||
|
7 months ago
|
for _, input := range r.inputs {
|
||
|
|
input.Close()
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
|
// Inputs returns the inputs of the pipeline.
|
||
|
4 months ago
|
func (r *rangeAggregationPipeline) Inputs() []Pipeline {
|
||
|
7 months ago
|
return r.inputs
|
||
|
|
}
|
||
|
|
|
||
|
|
// Transport returns the type of transport of the implementation.
|
||
|
4 months ago
|
func (r *rangeAggregationPipeline) Transport() Transport {
|
||
|
7 months ago
|
return Local
|
||
|
|
}
|
||
|
4 months ago
|
|
||
|
|
func newMatcherFactoryFromOpts(opts rangeAggregationOptions) *matcherFactory {
|
||
|
|
return &matcherFactory{
|
||
|
|
start: opts.startTs,
|
||
|
|
step: opts.step,
|
||
|
|
interval: opts.rangeInterval,
|
||
|
|
bounds: window{
|
||
|
|
start: opts.startTs.Add(-opts.rangeInterval),
|
||
|
|
end: opts.endTs,
|
||
|
|
},
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
|
type matcherFactory struct {
|
||
|
|
start time.Time
|
||
|
|
step time.Duration
|
||
|
|
interval time.Duration
|
||
|
|
bounds window
|
||
|
|
}
|
||
|
|
|
||
|
|
func (f *matcherFactory) createMatcher(windows []window) timestampMatchingWindowsFunc {
|
||
|
|
switch {
|
||
|
|
case f.step == 0:
|
||
|
|
// For instant queries, step == 0, meaning that all samples fall into the one and same step.
|
||
|
|
// A sample timestamp will always match the only time window available, unless the timestamp it out of range.
|
||
|
|
return f.createExactMatcher(windows)
|
||
|
|
case f.step == f.interval:
|
||
|
|
// If the step is equal to the range interval (e.g. when used $__auto in Grafana), then a sample timestamp matches exactly one time window.
|
||
|
|
return f.createAlignedMatcher(windows)
|
||
|
|
case f.step > f.interval:
|
||
|
|
// If the step is greater than the range interval, then a sample timestamp matches either one time window or no time window (and will be discarded).
|
||
|
|
return f.createGappedMatcher(windows)
|
||
|
|
case f.step < f.interval:
|
||
|
|
// If the step is smaller than the range interval, then a sample timestamp matches either one or multiple time windows.
|
||
|
|
return f.createOverlappingMatcher(windows)
|
||
|
|
default:
|
||
|
|
panic("invalid step and range interval")
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
|
// createExactMatcher is used for instant queries.
|
||
|
|
// The function returns a matcher that always returns the first aggregation window from the given windows if the timestamp is not out of range.
|
||
|
|
// It is expected that len(windows) is exactly 1, but it is not enforced.
|
||
|
|
//
|
||
|
|
// steps |---------x-------|
|
||
|
|
// interval |---------x-------|
|
||
|
|
func (f *matcherFactory) createExactMatcher(windows []window) timestampMatchingWindowsFunc {
|
||
|
|
return func(t time.Time) []window {
|
||
|
|
if !f.bounds.Contains(t) {
|
||
|
|
return nil // out of range
|
||
|
|
}
|
||
|
|
if len(windows) == 0 {
|
||
|
|
return nil
|
||
|
|
}
|
||
|
|
return []window{windows[0]}
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
|
// createAlignedMatcher is used for range queries.
|
||
|
|
// The function returns a matcher that always returns exactly one aggregation window that matches the timestamp if the timestamp is not out of range.
|
||
|
|
//
|
||
|
|
// steps |-----|---x-|-----|
|
||
|
|
// interval |-----|
|
||
|
|
// interval |---x-|
|
||
|
|
// interval |-----|
|
||
|
|
func (f *matcherFactory) createAlignedMatcher(windows []window) timestampMatchingWindowsFunc {
|
||
|
|
startNs := f.start.UnixNano()
|
||
|
|
stepNs := f.step.Nanoseconds()
|
||
|
|
|
||
|
|
return func(t time.Time) []window {
|
||
|
|
if !f.bounds.Contains(t) {
|
||
|
|
return nil // out of range
|
||
|
|
}
|
||
|
|
|
||
|
|
tNs := t.UnixNano()
|
||
|
|
// valid timestamps for window i: t > startNs + (i-1) * intervalNs && t <= startNs + i * intervalNs
|
||
|
|
windowIndex := (tNs - startNs + stepNs - 1) / stepNs // subtract 1ns because we are calculating 0-based indexes
|
||
|
|
return []window{windows[windowIndex]}
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
|
// createGappedMatcher is used for range queries.
|
||
|
|
// The function returns a matcher that either returns exactly one aggregation window that matches the timestamp, or none,
|
||
|
|
// if the timestamp is out of bounds or within bounds, but is within a "gap" between the end of an interval and the beginning of the next interval.
|
||
|
|
//
|
||
|
|
// steps |-----|---x-|-----|
|
||
|
|
// interval |--|
|
||
|
|
// interval |x-|
|
||
|
|
// interval |--|
|
||
|
|
func (f *matcherFactory) createGappedMatcher(windows []window) timestampMatchingWindowsFunc {
|
||
|
|
startNs := f.start.UnixNano()
|
||
|
|
stepNs := f.step.Nanoseconds()
|
||
|
|
|
||
|
|
return func(t time.Time) []window {
|
||
|
|
if !f.bounds.Contains(t) {
|
||
|
|
return nil // out of range
|
||
|
|
}
|
||
|
|
|
||
|
|
tNs := t.UnixNano()
|
||
|
|
// For gapped windows, window i covers: (start + i*step - interval, start + i*step]
|
||
|
|
windowIndex := (tNs - startNs + stepNs - 1) / stepNs // subtract 1ns because we are calculating 0-based indexes
|
||
|
|
matchingWindow := windows[windowIndex]
|
||
|
|
|
||
|
|
// Verify the timestamp is within the window (not in a gap)
|
||
|
|
if tNs > matchingWindow.start.UnixNano() {
|
||
|
|
return []window{matchingWindow}
|
||
|
|
}
|
||
|
|
|
||
|
|
return nil // timestamp is in a gap
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
|
// createOverlappingMatcher is used for range queries.
|
||
|
|
// The function returns a matcher that returns one or more aggregation windows that match the timestamp, if the timestamp is not out of range.
|
||
|
|
//
|
||
|
|
// steps |-----|---x-|-----|
|
||
|
|
// interval |x-------|
|
||
|
|
// interval |------x-|
|
||
|
|
// interval |--------|
|
||
|
|
func (f *matcherFactory) createOverlappingMatcher(windows []window) timestampMatchingWindowsFunc {
|
||
|
|
return func(t time.Time) []window {
|
||
|
|
if !f.bounds.Contains(t) {
|
||
|
|
return nil // out of range
|
||
|
|
}
|
||
|
|
|
||
|
|
// Find the last window that could contain the timestamp.
|
||
|
|
// We need to find the last window where t > window.startTs
|
||
|
|
// so search for the first window where t <= window.startTs
|
||
|
|
firstOOBIndex := sort.Search(len(windows), func(i int) bool {
|
||
|
|
return t.Compare(windows[i].start) <= 0
|
||
|
|
})
|
||
|
|
|
||
|
|
windowIndex := firstOOBIndex - 1
|
||
|
|
if windowIndex < 0 {
|
||
|
|
return nil
|
||
|
|
}
|
||
|
|
|
||
|
|
// Iterate backwards from last matching window to find all matches
|
||
|
|
var result []window
|
||
|
|
for _, window := range slices.Backward(windows[:windowIndex+1]) {
|
||
|
|
if t.Compare(window.start) > 0 && t.Compare(window.end) <= 0 {
|
||
|
|
result = append(result, window)
|
||
|
|
} else if t.Compare(window.end) > 0 {
|
||
|
|
// we've gone past all possible matches
|
||
|
|
break
|
||
|
|
}
|
||
|
|
}
|
||
|
|
|
||
|
|
return result
|
||
|
|
}
|
||
|
|
}
|