Like Prometheus, but for logs.
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.
loki/pkg/engine/executor/executor.go

192 lines
4.8 KiB

package executor
import (
"context"
"errors"
"fmt"
"github.com/thanos-io/objstore"
"github.com/grafana/loki/v3/pkg/dataobj"
refactor(dataobj): invert dependency between dataobj and sections (#17762) Originally, the dataobj package was a higher-level API around sections. This design caused it to become a bottleneck: * Implementing any new public behaviour for a section required bubbling it up to the dataobj API for it to be exposed, making it tedious to add new sections or update existing ones. * The `dataobj.Builder` pattern was focused on constructing dataobjs for storing log data, which will cause friction as we build objects around other use cases. This PR builds on top of the foundation laid out by #17704 and #17708, fully inverting the dependency between dataobj and sections: * The `dataobj` package has no knowledge of what sections exist, and can now be used for writing and reading generic sections. Section packages now create higher-level APIs around the abstractions provided by `dataobj`. * Section packages are now public, and callers interact directly with these packages for writing and reading section-specific data. * All logic for a section (encoding, decoding, buffering, reading) is now fully self-contained inside the section package. Previously, the implementation of each section was spread across three packages (`pkg/dataobj/internal/encoding`, `pkg/dataobj/internal/sections/SECTION`, `pkg/dataobj`). * Cutting a section is now a decision made by the caller rather than the section implementation. Previously, the logs section builder would create multiple sections. For the most part, this change is a no-op, with two exceptions: 1. Section cutting is now performed by the caller; however, this shouldn't result in any issues. 2. Removing the high-level `dataobj.Stream` and `dataobj.Record` types will temporarily reduce the allocation gains from #16988. I will address this after this PR is merged.
8 months ago
"github.com/grafana/loki/v3/pkg/dataobj/sections/logs"
"github.com/grafana/loki/v3/pkg/engine/planner/physical"
)
type Config struct {
BatchSize int64
Bucket objstore.Bucket
}
func Run(ctx context.Context, cfg Config, plan *physical.Plan) Pipeline {
c := &Context{
plan: plan,
batchSize: cfg.BatchSize,
bucket: cfg.Bucket,
}
if plan == nil {
return errorPipeline(errors.New("plan is nil"))
}
node, err := plan.Root()
if err != nil {
return errorPipeline(err)
}
return c.execute(ctx, node)
}
// Context is the execution context
type Context struct {
batchSize int64
plan *physical.Plan
evaluator expressionEvaluator
bucket objstore.Bucket
}
func (c *Context) execute(ctx context.Context, node physical.Node) Pipeline {
children := c.plan.Children(node)
inputs := make([]Pipeline, 0, len(children))
for _, child := range children {
inputs = append(inputs, c.execute(ctx, child))
}
switch n := node.(type) {
case *physical.DataObjScan:
return c.executeDataObjScan(ctx, n)
case *physical.SortMerge:
return c.executeSortMerge(ctx, n, inputs)
case *physical.Limit:
return c.executeLimit(ctx, n, inputs)
case *physical.Filter:
return c.executeFilter(ctx, n, inputs)
case *physical.Projection:
return c.executeProjection(ctx, n, inputs)
case *physical.RangeAggregation:
return c.executeRangeAggregation(ctx, n, inputs)
case *physical.VectorAggregation:
return c.executeVectorAggregation(ctx, n, inputs)
default:
return errorPipeline(fmt.Errorf("invalid node type: %T", node))
}
}
func (c *Context) executeDataObjScan(ctx context.Context, node *physical.DataObjScan) Pipeline {
if c.bucket == nil {
return errorPipeline(errors.New("no object store bucket configured"))
}
refactor(dataobj): invert dependency between dataobj and sections (#17762) Originally, the dataobj package was a higher-level API around sections. This design caused it to become a bottleneck: * Implementing any new public behaviour for a section required bubbling it up to the dataobj API for it to be exposed, making it tedious to add new sections or update existing ones. * The `dataobj.Builder` pattern was focused on constructing dataobjs for storing log data, which will cause friction as we build objects around other use cases. This PR builds on top of the foundation laid out by #17704 and #17708, fully inverting the dependency between dataobj and sections: * The `dataobj` package has no knowledge of what sections exist, and can now be used for writing and reading generic sections. Section packages now create higher-level APIs around the abstractions provided by `dataobj`. * Section packages are now public, and callers interact directly with these packages for writing and reading section-specific data. * All logic for a section (encoding, decoding, buffering, reading) is now fully self-contained inside the section package. Previously, the implementation of each section was spread across three packages (`pkg/dataobj/internal/encoding`, `pkg/dataobj/internal/sections/SECTION`, `pkg/dataobj`). * Cutting a section is now a decision made by the caller rather than the section implementation. Previously, the logs section builder would create multiple sections. For the most part, this change is a no-op, with two exceptions: 1. Section cutting is now performed by the caller; however, this shouldn't result in any issues. 2. Removing the high-level `dataobj.Stream` and `dataobj.Record` types will temporarily reduce the allocation gains from #16988. I will address this after this PR is merged.
8 months ago
predicates := make([]logs.RowPredicate, 0, len(node.Predicates))
for _, p := range node.Predicates {
conv, err := buildLogsPredicate(p)
if err != nil {
return errorPipeline(err)
}
predicates = append(predicates, conv)
}
refactor(dataobj): invert dependency between dataobj and sections (#17762) Originally, the dataobj package was a higher-level API around sections. This design caused it to become a bottleneck: * Implementing any new public behaviour for a section required bubbling it up to the dataobj API for it to be exposed, making it tedious to add new sections or update existing ones. * The `dataobj.Builder` pattern was focused on constructing dataobjs for storing log data, which will cause friction as we build objects around other use cases. This PR builds on top of the foundation laid out by #17704 and #17708, fully inverting the dependency between dataobj and sections: * The `dataobj` package has no knowledge of what sections exist, and can now be used for writing and reading generic sections. Section packages now create higher-level APIs around the abstractions provided by `dataobj`. * Section packages are now public, and callers interact directly with these packages for writing and reading section-specific data. * All logic for a section (encoding, decoding, buffering, reading) is now fully self-contained inside the section package. Previously, the implementation of each section was spread across three packages (`pkg/dataobj/internal/encoding`, `pkg/dataobj/internal/sections/SECTION`, `pkg/dataobj`). * Cutting a section is now a decision made by the caller rather than the section implementation. Previously, the logs section builder would create multiple sections. For the most part, this change is a no-op, with two exceptions: 1. Section cutting is now performed by the caller; however, this shouldn't result in any issues. 2. Removing the high-level `dataobj.Stream` and `dataobj.Record` types will temporarily reduce the allocation gains from #16988. I will address this after this PR is merged.
8 months ago
obj, err := dataobj.FromBucket(ctx, c.bucket, string(node.Location))
if err != nil {
return errorPipeline(fmt.Errorf("creating data object: %w", err))
}
return newDataobjScanPipeline(ctx, dataobjScanOptions{
refactor(dataobj): invert dependency between dataobj and sections (#17762) Originally, the dataobj package was a higher-level API around sections. This design caused it to become a bottleneck: * Implementing any new public behaviour for a section required bubbling it up to the dataobj API for it to be exposed, making it tedious to add new sections or update existing ones. * The `dataobj.Builder` pattern was focused on constructing dataobjs for storing log data, which will cause friction as we build objects around other use cases. This PR builds on top of the foundation laid out by #17704 and #17708, fully inverting the dependency between dataobj and sections: * The `dataobj` package has no knowledge of what sections exist, and can now be used for writing and reading generic sections. Section packages now create higher-level APIs around the abstractions provided by `dataobj`. * Section packages are now public, and callers interact directly with these packages for writing and reading section-specific data. * All logic for a section (encoding, decoding, buffering, reading) is now fully self-contained inside the section package. Previously, the implementation of each section was spread across three packages (`pkg/dataobj/internal/encoding`, `pkg/dataobj/internal/sections/SECTION`, `pkg/dataobj`). * Cutting a section is now a decision made by the caller rather than the section implementation. Previously, the logs section builder would create multiple sections. For the most part, this change is a no-op, with two exceptions: 1. Section cutting is now performed by the caller; however, this shouldn't result in any issues. 2. Removing the high-level `dataobj.Stream` and `dataobj.Record` types will temporarily reduce the allocation gains from #16988. I will address this after this PR is merged.
8 months ago
Object: obj,
StreamIDs: node.StreamIDs,
chore(engine): Use shard information from query frontend in the new query engine (#17792) **What this PR does / why we need it**: This PR introduces the sharding to the new query engine to be able to test the engine using the exsiting Loki architecture with query frontend, query scheduler, and queriers. **Note, this is only an interim solution used for validating and testing.** By design, the first phase of the query engine implementation does only local execution of queries without sharding or time-based splitting. However, for testing the results of this first phase, we can utilise dual-ingestion (writing both chunks/tsdb and data objects in a way that sharding and splitting is performed in the query frontend using the existing middlewares. On the queriers themselves, the sub-queries are parsed and planned using the new logical and physical planner. If the query can successfully be planned, it will be executed by the new engine, otherwise it falls back to the old engine. **Shard size considerations** While performing time-splitting in the frontend works for data objects as well, sharding by information from TSDB is not directly mappable to data objects. The default target shard size in TSDB is 600MB (decompressed), whereas target size of data objects is 1GB compressed or roughly 10-15GB uncompressed. However, individual logs sections of a data object have a target size of 128MB, which is roughly 0.9-1.2GB. That is 1.5-2x larger than the TSDB target shard size. So when using the sharding calculation from TSDB, it would over-shard for data object sections, which is likely acceptable for testing and good enough for proving that local execution with the new engine works. **How does sharding with data objects in this PR work?** The query frontend passes down the calculated shards as part of the query parameters of the serialised sub-request. The logical planner on the querier stores the Shard annotation on the `MAKETABLE` alongside the stream selector. This is then used by the physical planner to filter out only the relevant sections of the resolved data objects from the metastore lookup. During exeuction, only readers for the relevant sections are initialised when performing the `DataObjScan`. Signed-off-by: Christian Haudum <christian.haudum@gmail.com> Co-authored-by: Ashwanth <iamashwanth@gmail.com>
7 months ago
Sections: node.Sections,
Predicates: predicates,
Projections: node.Projections,
Direction: node.Direction,
Limit: node.Limit,
})
}
chore(engine): Implement execution pipeline for SortMerge operator (#17406) This PR contains an implementation of the k-way merge operation without using a heap, like @rfratto described [here](https://github.com/grafana/loki/pull/17280). The SortMerge is implemented only using slices: * Maintain the following invariant: * For each input pipeline, we store the next record to process. (this already exists as `HeapSortMerge.batches`) * Additionally for each record, track the starting slice offset (which resets to zero whenever a new record is loaded in). * Iteration stops when all input pipelines have been exhausted (no change from how this is now). * To get the next record: * Iterate through each record, looking at the value from their starting slice offset. * Track the top _two_ winners (e.g., the record whose next value is the smallest and the record whose next value is the next smallest). * Find the largest offset in the starting record whose value is still less than the value of the runner-up record from the previous step. * Return the slice of that record using the two offsets, and update the stored offset of the returned record for the next call to `Read`. This approach, like the one with heap, still requires to concatenate (coalesce) the single row records - which is not implemented in this PR yet. On that note, single row records are the worst case scenario with this implementation, not necessarily the regular case. **Update:** After an offline discussion, @owen-d and I agreed on ignoring the worst-case scenario of single-row records for now. Signed-off-by: Christian Haudum <christian.haudum@gmail.com>
9 months ago
func (c *Context) executeSortMerge(_ context.Context, sortmerge *physical.SortMerge, inputs []Pipeline) Pipeline {
if len(inputs) == 0 {
return emptyPipeline()
}
chore(engine): Implement execution pipeline for SortMerge operator (#17406) This PR contains an implementation of the k-way merge operation without using a heap, like @rfratto described [here](https://github.com/grafana/loki/pull/17280). The SortMerge is implemented only using slices: * Maintain the following invariant: * For each input pipeline, we store the next record to process. (this already exists as `HeapSortMerge.batches`) * Additionally for each record, track the starting slice offset (which resets to zero whenever a new record is loaded in). * Iteration stops when all input pipelines have been exhausted (no change from how this is now). * To get the next record: * Iterate through each record, looking at the value from their starting slice offset. * Track the top _two_ winners (e.g., the record whose next value is the smallest and the record whose next value is the next smallest). * Find the largest offset in the starting record whose value is still less than the value of the runner-up record from the previous step. * Return the slice of that record using the two offsets, and update the stored offset of the returned record for the next call to `Read`. This approach, like the one with heap, still requires to concatenate (coalesce) the single row records - which is not implemented in this PR yet. On that note, single row records are the worst case scenario with this implementation, not necessarily the regular case. **Update:** After an offline discussion, @owen-d and I agreed on ignoring the worst-case scenario of single-row records for now. Signed-off-by: Christian Haudum <christian.haudum@gmail.com>
9 months ago
pipeline, err := NewSortMergePipeline(inputs, sortmerge.Order, sortmerge.Column, c.evaluator)
if err != nil {
return errorPipeline(err)
}
return pipeline
}
func (c *Context) executeLimit(_ context.Context, limit *physical.Limit, inputs []Pipeline) Pipeline {
if len(inputs) == 0 {
return emptyPipeline()
}
if len(inputs) > 1 {
return errorPipeline(fmt.Errorf("limit expects exactly one input, got %d", len(inputs)))
}
return NewLimitPipeline(inputs[0], limit.Skip, limit.Fetch)
}
func (c *Context) executeFilter(_ context.Context, filter *physical.Filter, inputs []Pipeline) Pipeline {
if len(inputs) == 0 {
return emptyPipeline()
}
// TODO: support multiple inputs
if len(inputs) > 1 {
return errorPipeline(fmt.Errorf("filter expects exactly one input, got %d", len(inputs)))
}
return NewFilterPipeline(filter, inputs[0], c.evaluator)
}
func (c *Context) executeProjection(_ context.Context, proj *physical.Projection, inputs []Pipeline) Pipeline {
if len(inputs) == 0 {
return emptyPipeline()
}
if len(inputs) > 1 {
// unsupported for now
return errorPipeline(fmt.Errorf("projection expects exactly one input, got %d", len(inputs)))
}
if len(proj.Columns) == 0 {
return errorPipeline(fmt.Errorf("projection expects at least one column, got 0"))
}
p, err := NewProjectPipeline(inputs[0], proj.Columns, &c.evaluator)
if err != nil {
return errorPipeline(err)
}
return p
}
func (c *Context) executeRangeAggregation(_ context.Context, plan *physical.RangeAggregation, inputs []Pipeline) Pipeline {
if len(inputs) == 0 {
return emptyPipeline()
}
pipeline, err := NewRangeAggregationPipeline(inputs, c.evaluator, rangeAggregationOptions{
partitionBy: plan.PartitionBy,
startTs: plan.Start,
endTs: plan.End,
rangeInterval: plan.Range,
step: plan.Step,
})
if err != nil {
return errorPipeline(err)
}
return pipeline
}
func (c *Context) executeVectorAggregation(_ context.Context, plan *physical.VectorAggregation, inputs []Pipeline) Pipeline {
if len(inputs) == 0 {
return emptyPipeline()
}
pipeline, err := NewVectorAggregationPipeline(inputs, plan.GroupBy, c.evaluator)
if err != nil {
return errorPipeline(err)
}
return pipeline
}