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/internal/worker/worker_test.go

188 lines
5.4 KiB

package worker_test
import (
"context"
"errors"
"fmt"
"os"
"testing"
"time"
"github.com/apache/arrow-go/v18/arrow"
"github.com/apache/arrow-go/v18/arrow/array"
"github.com/apache/arrow-go/v18/arrow/memory"
"github.com/go-kit/log"
"github.com/grafana/dskit/services"
"github.com/grafana/dskit/user"
"github.com/prometheus/client_golang/prometheus"
"github.com/stretchr/testify/require"
"github.com/thanos-io/objstore"
"github.com/grafana/loki/v3/pkg/dataobj/metastore"
"github.com/grafana/loki/v3/pkg/engine/internal/executor"
"github.com/grafana/loki/v3/pkg/engine/internal/planner/logical"
"github.com/grafana/loki/v3/pkg/engine/internal/planner/physical"
"github.com/grafana/loki/v3/pkg/engine/internal/scheduler"
"github.com/grafana/loki/v3/pkg/engine/internal/util/objtest"
"github.com/grafana/loki/v3/pkg/engine/internal/worker"
"github.com/grafana/loki/v3/pkg/engine/internal/workflow"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql"
"github.com/grafana/loki/v3/pkg/util/arrowtest"
)
// Test runs an end-to-end test of the worker.
func Test(t *testing.T) {
builder := objtest.NewBuilder(t)
logger := log.NewNopLogger()
if testing.Verbose() {
logger = log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr))
}
sched := newTestScheduler(t, logger)
_ = newTestWorker(t, logger, builder.Location(), sched)
ctx := user.InjectOrgID(t.Context(), objtest.Tenant)
builder.Append(ctx, logproto.Stream{
Labels: `{app="loki", env="dev"}`,
Entries: []logproto.Entry{{
Timestamp: time.Date(2025, time.January, 1, 0, 0, 0, 0, time.UTC),
Line: "Hello, world!",
}, {
Timestamp: time.Date(2025, time.January, 1, 0, 0, 1, 0, time.UTC),
Line: "Goodbye, world!",
}},
})
builder.Close()
params, err := logql.NewLiteralParams(
`{app="loki"}`,
time.Date(2025, time.January, 1, 0, 0, 0, 0, time.UTC),
time.Date(2025, time.January, 2, 0, 0, 0, 0, time.UTC),
0,
0,
logproto.BACKWARD,
1000,
[]string{"0_of_1"},
nil,
)
require.NoError(t, err, "expected to be able to create literal LogQL params")
wf := buildWorkflow(ctx, t, logger, builder.Location(), sched, params)
pipeline, err := wf.Run(ctx)
require.NoError(t, err)
expected := arrowtest.Rows{
{
"timestamp_ns.builtin.timestamp": time.Date(2025, time.January, 1, 0, 0, 1, 0, time.UTC),
"utf8.label.app": "loki",
"utf8.label.env": "dev",
"utf8.builtin.message": "Goodbye, world!",
},
{
"timestamp_ns.builtin.timestamp": time.Date(2025, time.January, 1, 0, 0, 0, 0, time.UTC),
"utf8.label.app": "loki",
"utf8.label.env": "dev",
"utf8.builtin.message": "Hello, world!",
},
}
actual, err := arrowtest.TableRows(memory.DefaultAllocator, readTable(ctx, t, pipeline))
require.NoError(t, err, "failed to get rows from table")
require.Equal(t, expected, actual)
}
func newTestScheduler(t *testing.T, logger log.Logger) *scheduler.Scheduler {
t.Helper()
sched, err := scheduler.New(scheduler.Config{Logger: logger})
require.NoError(t, err)
require.NoError(t, services.StartAndAwaitRunning(t.Context(), sched.Service()))
t.Cleanup(func() {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
require.NoError(t, services.StopAndAwaitTerminated(ctx, sched.Service()))
})
return sched
}
func newTestWorker(t *testing.T, logger log.Logger, loc objtest.Location, sched *scheduler.Scheduler) *worker.Worker {
t.Helper()
w, err := worker.New(worker.Config{
Logger: logger,
Bucket: loc.Bucket,
LocalScheduler: sched,
BatchSize: 2048,
// Create enough threads to guarantee all tasks can be scheduled without
// blocking.
NumThreads: 8,
})
require.NoError(t, err, "expected to create worker")
require.NoError(t, services.StartAndAwaitRunning(t.Context(), w.Service()))
t.Cleanup(func() {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
require.NoError(t, services.StopAndAwaitTerminated(ctx, w.Service()))
})
return w
}
func buildWorkflow(ctx context.Context, t *testing.T, logger log.Logger, loc objtest.Location, sched *scheduler.Scheduler, params logql.Params) *workflow.Workflow {
logicalPlan, err := logical.BuildPlan(params)
require.NoError(t, err, "expected to create logical plan")
ms := metastore.NewObjectMetastore(
objstore.NewPrefixedBucket(loc.Bucket, loc.IndexPrefix),
logger,
prometheus.NewRegistry(),
)
catalog := physical.NewMetastoreCatalog(ctx, ms)
planner := physical.NewPlanner(physical.NewContext(params.Start(), params.End()), catalog)
plan, err := planner.Build(logicalPlan)
require.NoError(t, err, "expected to create physical plan")
plan, err = planner.Optimize(plan)
require.NoError(t, err, "expected to optimize physical plan")
if testing.Verbose() {
fmt.Fprintln(os.Stderr, physical.PrintAsTree(plan))
}
wf, err := workflow.New(logger, objtest.Tenant, sched, plan)
require.NoError(t, err)
if testing.Verbose() {
workflow.Fprint(os.Stderr, wf)
}
return wf
}
func readTable(ctx context.Context, t *testing.T, p executor.Pipeline) arrow.Table {
var recs []arrow.Record
for {
rec, err := p.Read(ctx)
if rec != nil {
if rec.NumRows() > 0 {
recs = append(recs, rec)
}
}
if err != nil && errors.Is(err, executor.EOF) {
break
}
require.NoError(t, err)
}
return array.NewTableFromRecords(recs[0].Schema(), recs)
}