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/storage/batch_test.go

906 lines
19 KiB

package storage
import (
"context"
"fmt"
"testing"
"time"
"github.com/cortexproject/cortex/pkg/chunk"
"github.com/pkg/errors"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/promql"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/user"
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/stats"
)
func Test_newBatchChunkIterator(t *testing.T) {
tests := map[string]struct {
chunks []*LazyChunk
expected []logproto.Stream
matchers string
start, end time.Time
direction logproto.Direction
batchSize int
}{
"forward with overlap": {
[]*LazyChunk{
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
{
Timestamp: from.Add(4 * time.Millisecond),
Line: "5",
},
},
}),
},
[]logproto.Stream{
{
Labels: fooLabels,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
},
},
fooLabelsWithName,
from, from.Add(4 * time.Millisecond),
logproto.FORWARD,
2,
},
"forward with overlapping non-continuous entries": {
[]*LazyChunk{
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
},
[]logproto.Stream{
{
Labels: fooLabels,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
},
},
fooLabelsWithName,
from, from.Add(3 * time.Millisecond),
logproto.FORWARD,
2,
},
"backward with overlap": {
[]*LazyChunk{
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
{
Timestamp: from.Add(4 * time.Millisecond),
Line: "5",
},
},
}),
},
[]logproto.Stream{
{
Labels: fooLabels,
Entries: []logproto.Entry{
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from,
Line: "1",
},
},
},
},
fooLabelsWithName,
from, from.Add(4 * time.Millisecond),
logproto.BACKWARD,
2,
},
"backward with overlapping non-continuous entries": {
[]*LazyChunk{
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(0 * time.Millisecond),
Line: "0",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "3",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(1 * time.Millisecond),
Line: "1",
},
{
Timestamp: from.Add(6 * time.Millisecond),
Line: "6",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(5 * time.Millisecond),
Line: "5",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(4 * time.Millisecond),
Line: "4",
},
{
Timestamp: from.Add(7 * time.Millisecond),
Line: "7",
},
},
}),
},
[]logproto.Stream{
{
Labels: fooLabels,
Entries: []logproto.Entry{
{
Timestamp: from.Add(7 * time.Millisecond),
Line: "7",
},
{
Timestamp: from.Add(6 * time.Millisecond),
Line: "6",
},
{
Timestamp: from.Add(5 * time.Millisecond),
Line: "5",
},
{
Timestamp: from.Add(4 * time.Millisecond),
Line: "4",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(1 * time.Millisecond),
Line: "1",
},
{
Timestamp: from.Add(0 * time.Millisecond),
Line: "0",
},
},
},
},
fooLabelsWithName,
from, from.Add(8 * time.Millisecond),
logproto.BACKWARD,
2,
},
"forward without overlap": {
[]*LazyChunk{
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
},
[]logproto.Stream{
{
Labels: fooLabels,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
},
},
fooLabelsWithName,
from, from.Add(3 * time.Millisecond),
logproto.FORWARD,
2,
},
"backward without overlap": {
[]*LazyChunk{
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
}),
newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
}),
},
[]logproto.Stream{
{
Labels: fooLabels,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from,
Line: "1",
},
},
},
},
fooLabelsWithName,
from, from.Add(3 * time.Millisecond),
logproto.BACKWARD,
2,
},
}
for name, tt := range tests {
tt := tt
t.Run(name, func(t *testing.T) {
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
5 years ago
it := newBatchChunkIterator(context.Background(), tt.chunks, tt.batchSize, newMatchers(tt.matchers), nil, newQuery("", tt.start, tt.end, tt.direction, nil))
streams, _, err := iter.ReadBatch(it, 1000)
_ = it.Close()
if err != nil {
t.Fatalf("error reading batch %s", err)
}
assertStream(t, tt.expected, streams.Streams)
})
}
}
func TestPartitionOverlappingchunks(t *testing.T) {
var (
oneThroughFour = newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
},
})
two = newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(1 * time.Millisecond),
Line: "2",
},
},
})
three = newLazyChunk(logproto.Stream{
Labels: fooLabelsWithName,
Entries: []logproto.Entry{
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
})
)
for i, tc := range []struct {
input []*LazyChunk
expected [][]*LazyChunk
}{
{
input: []*LazyChunk{
oneThroughFour,
two,
three,
},
expected: [][]*LazyChunk{
{oneThroughFour},
{two, three},
},
},
{
input: []*LazyChunk{
two,
oneThroughFour,
three,
},
expected: [][]*LazyChunk{
{oneThroughFour},
{two, three},
},
},
{
input: []*LazyChunk{
two,
two,
three,
three,
},
expected: [][]*LazyChunk{
{two, three},
{two, three},
},
},
} {
t.Run(fmt.Sprintf("%d", i), func(t *testing.T) {
out := partitionOverlappingChunks(tc.input)
require.Equal(t, tc.expected, out)
})
}
}
func TestBuildHeapIterator(t *testing.T) {
var (
firstChunk = newLazyChunk(logproto.Stream{
Labels: "{foo=\"bar\"}",
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
},
})
secondChunk = newLazyInvalidChunk(logproto.Stream{
Labels: "{foo=\"bar\"}",
Entries: []logproto.Entry{
{
Timestamp: from.Add(3 * time.Millisecond),
Line: "4",
},
{
Timestamp: from.Add(4 * time.Millisecond),
Line: "5",
},
},
})
thirdChunk = newLazyChunk(logproto.Stream{
Labels: "{foo=\"bar\"}",
Entries: []logproto.Entry{
{
Timestamp: from.Add(5 * time.Millisecond),
Line: "6",
},
},
})
)
for i, tc := range []struct {
input [][]*LazyChunk
expected []logproto.Stream
}{
{
[][]*LazyChunk{
{firstChunk},
{thirdChunk},
},
[]logproto.Stream{
{
Labels: "{foo=\"bar\"}",
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(5 * time.Millisecond),
Line: "6",
},
},
},
},
},
{
[][]*LazyChunk{
{secondChunk},
{firstChunk, thirdChunk},
},
[]logproto.Stream{
{
Labels: "{foo=\"bar\"}",
Entries: []logproto.Entry{
{
Timestamp: from,
Line: "1",
},
{
Timestamp: from.Add(time.Millisecond),
Line: "2",
},
{
Timestamp: from.Add(2 * time.Millisecond),
Line: "3",
},
{
Timestamp: from.Add(5 * time.Millisecond),
Line: "6",
},
},
},
},
},
} {
t.Run(fmt.Sprintf("%d", i), func(t *testing.T) {
ctx = user.InjectOrgID(context.Background(), "test-user")
b := &batchChunkIterator{
ctx: ctx,
req: &logproto.QueryRequest{Direction: logproto.FORWARD},
labels: map[model.Fingerprint]string{},
}
it, err := b.buildHeapIterator(tc.input, from, from.Add(6*time.Millisecond), nil)
if err != nil {
t.Errorf("buildHeapIterator error = %v", err)
return
}
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
5 years ago
req := newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil)
streams, _, err := iter.ReadBatch(it, req.Limit)
_ = it.Close()
if err != nil {
t.Fatalf("error reading batch %s", err)
}
assertStream(t, tc.expected, streams.Streams)
})
}
}
func TestDropLabels(t *testing.T) {
for i, tc := range []struct {
ls labels.Labels
drop []string
expected labels.Labels
}{
{
ls: labels.Labels{
labels.Label{
Name: "a",
Value: "1",
},
labels.Label{
Name: "b",
Value: "2",
},
labels.Label{
Name: "c",
Value: "3",
},
},
drop: []string{"b"},
expected: labels.Labels{
labels.Label{
Name: "a",
Value: "1",
},
labels.Label{
Name: "c",
Value: "3",
},
},
},
} {
t.Run(fmt.Sprintf("%d", i), func(t *testing.T) {
dropped := dropLabels(tc.ls, tc.drop...)
require.Equal(t, tc.expected, dropped)
})
}
}
func Test_IsInvalidChunkError(t *testing.T) {
tests := []struct {
name string
err error
expectedResult bool
}{
{
"invalid chunk cheksum error from cortex",
promql.ErrStorage{Err: chunk.ErrInvalidChecksum},
true,
},
{
"invalid chunk cheksum error from loki",
promql.ErrStorage{Err: chunkenc.ErrInvalidChecksum},
true,
},
{
"cache error",
promql.ErrStorage{Err: errors.New("error fetching from cache")},
false,
},
{
"no error from cortex or loki",
nil,
false,
},
}
for _, tc := range tests {
result := isInvalidChunkError(tc.err)
require.Equal(t, tc.expectedResult, result)
}
}
var entry logproto.Entry
func Benchmark_store_OverlappingChunks(b *testing.B) {
b.ReportAllocs()
st := &store{
cfg: Config{
MaxChunkBatchSize: 50,
},
Store: newMockChunkStore(newOverlappingStreams(200, 200)),
}
b.ResetTimer()
ctx := user.InjectOrgID(stats.NewContext(context.Background()), "fake")
start := time.Now()
for i := 0; i < b.N; i++ {
it, err := st.LazyQuery(ctx, logql.SelectParams{QueryRequest: &logproto.QueryRequest{
Selector: `{foo="bar"}`,
Direction: logproto.BACKWARD,
Limit: 0,
Shards: nil,
Start: time.Unix(0, 1),
End: time.Unix(0, time.Now().UnixNano()),
}})
if err != nil {
b.Fatal(err)
}
for it.Next() {
entry = it.Entry()
}
if err := it.Close(); err != nil {
b.Fatal(err)
}
}
r := stats.Snapshot(ctx, time.Since(start))
b.Log("Total chunks:" + fmt.Sprintf("%d", r.Store.TotalChunksRef))
b.Log("Total bytes decompressed:" + fmt.Sprintf("%d", r.Store.DecompressedBytes))
}
func newOverlappingStreams(streamCount int, entryCount int) []*logproto.Stream {
streams := make([]*logproto.Stream, streamCount)
for i := range streams {
streams[i] = &logproto.Stream{
Labels: fmt.Sprintf(`{foo="bar",id="%d"}`, i),
Entries: make([]logproto.Entry, entryCount),
}
for j := range streams[i].Entries {
streams[i].Entries[j] = logproto.Entry{
Timestamp: time.Unix(0, int64(1+j)),
Line: "a very compressible log line duh",
}
}
}
return streams
}