mirror of https://github.com/grafana/loki
Cache overlapping blocks (#2239)
* Caches block iteration that are overlapping together to avoid reprocessing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Final touch. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>pull/2247/head
parent
732fe6d14f
commit
6ab832b2ca
@ -1,31 +0,0 @@ |
||||
package chunkenc |
||||
|
||||
import ( |
||||
"context" |
||||
"errors" |
||||
"time" |
||||
|
||||
"github.com/cortexproject/cortex/pkg/chunk" |
||||
|
||||
"github.com/grafana/loki/pkg/iter" |
||||
"github.com/grafana/loki/pkg/logproto" |
||||
"github.com/grafana/loki/pkg/logql" |
||||
) |
||||
|
||||
// LazyChunk loads the chunk when it is accessed.
|
||||
type LazyChunk struct { |
||||
Chunk chunk.Chunk |
||||
IsValid bool |
||||
Fetcher *chunk.Fetcher |
||||
} |
||||
|
||||
// Iterator returns an entry iterator.
|
||||
func (c *LazyChunk) Iterator(ctx context.Context, from, through time.Time, direction logproto.Direction, filter logql.LineFilter) (iter.EntryIterator, error) { |
||||
// If the chunk is already loaded, then use that.
|
||||
if c.Chunk.Data != nil { |
||||
lokiChunk := c.Chunk.Data.(*Facade).LokiChunk() |
||||
return lokiChunk.Iterator(ctx, from, through, direction, filter) |
||||
} |
||||
|
||||
return nil, errors.New("chunk is not loaded") |
||||
} |
@ -0,0 +1,94 @@ |
||||
package storage |
||||
|
||||
import ( |
||||
"github.com/grafana/loki/pkg/iter" |
||||
"github.com/grafana/loki/pkg/logproto" |
||||
) |
||||
|
||||
// cachedIterator is an iterator that caches iteration to be replayed later on.
|
||||
type cachedIterator struct { |
||||
cache []*logproto.Entry |
||||
base iter.EntryIterator |
||||
|
||||
labels string |
||||
curr int |
||||
|
||||
closeErr error |
||||
iterErr error |
||||
} |
||||
|
||||
// newCachedIterator creates an iterator that cache iteration result and can be iterated again
|
||||
// after closing it without re-using the underlaying iterator `it`.
|
||||
// The cache iterator should be used for entries that belongs to the same stream only.
|
||||
func newCachedIterator(it iter.EntryIterator, cap int) *cachedIterator { |
||||
c := &cachedIterator{ |
||||
base: it, |
||||
cache: make([]*logproto.Entry, 0, cap), |
||||
curr: -1, |
||||
} |
||||
c.load() |
||||
return c |
||||
} |
||||
|
||||
func (it *cachedIterator) reset() { |
||||
it.curr = -1 |
||||
} |
||||
|
||||
func (it *cachedIterator) load() { |
||||
if it.base != nil { |
||||
defer func() { |
||||
it.closeErr = it.base.Close() |
||||
it.iterErr = it.base.Error() |
||||
it.base = nil |
||||
it.reset() |
||||
}() |
||||
// set labels using the first entry
|
||||
if !it.base.Next() { |
||||
return |
||||
} |
||||
it.labels = it.base.Labels() |
||||
|
||||
// add all entries until the base iterator is exhausted
|
||||
for { |
||||
e := it.base.Entry() |
||||
it.cache = append(it.cache, &e) |
||||
if !it.base.Next() { |
||||
break |
||||
} |
||||
} |
||||
|
||||
} |
||||
} |
||||
|
||||
func (it *cachedIterator) Next() bool { |
||||
if len(it.cache) == 0 { |
||||
it.cache = nil |
||||
return false |
||||
} |
||||
if it.curr+1 >= len(it.cache) { |
||||
return false |
||||
} |
||||
it.curr++ |
||||
return it.curr < len(it.cache) |
||||
} |
||||
|
||||
func (it *cachedIterator) Entry() logproto.Entry { |
||||
if len(it.cache) == 0 { |
||||
return logproto.Entry{} |
||||
} |
||||
if it.curr < 0 { |
||||
return *it.cache[0] |
||||
} |
||||
return *it.cache[it.curr] |
||||
} |
||||
|
||||
func (it *cachedIterator) Labels() string { |
||||
return it.labels |
||||
} |
||||
|
||||
func (it *cachedIterator) Error() error { return it.iterErr } |
||||
|
||||
func (it *cachedIterator) Close() error { |
||||
it.reset() |
||||
return it.closeErr |
||||
} |
@ -0,0 +1,86 @@ |
||||
package storage |
||||
|
||||
import ( |
||||
"errors" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/stretchr/testify/require" |
||||
|
||||
"github.com/grafana/loki/pkg/iter" |
||||
"github.com/grafana/loki/pkg/logproto" |
||||
) |
||||
|
||||
func Test_CachedIterator(t *testing.T) { |
||||
stream := logproto.Stream{ |
||||
Labels: `{foo="bar"}`, |
||||
Entries: []logproto.Entry{ |
||||
{Timestamp: time.Unix(0, 1), Line: "1"}, |
||||
{Timestamp: time.Unix(0, 2), Line: "2"}, |
||||
{Timestamp: time.Unix(0, 3), Line: "3"}, |
||||
}, |
||||
} |
||||
c := newCachedIterator(iter.NewStreamIterator(stream), 3) |
||||
|
||||
assert := func() { |
||||
// we should crash for call of entry without next although that's not expected.
|
||||
require.Equal(t, stream.Labels, c.Labels()) |
||||
require.Equal(t, stream.Entries[0], c.Entry()) |
||||
require.Equal(t, true, c.Next()) |
||||
require.Equal(t, stream.Entries[0], c.Entry()) |
||||
require.Equal(t, true, c.Next()) |
||||
require.Equal(t, stream.Entries[1], c.Entry()) |
||||
require.Equal(t, true, c.Next()) |
||||
require.Equal(t, stream.Entries[2], c.Entry()) |
||||
require.Equal(t, false, c.Next()) |
||||
require.Equal(t, nil, c.Error()) |
||||
require.Equal(t, stream.Entries[2], c.Entry()) |
||||
require.Equal(t, false, c.Next()) |
||||
} |
||||
|
||||
assert() |
||||
|
||||
// Close the iterator reset it to the beginning.
|
||||
require.Equal(t, nil, c.Close()) |
||||
|
||||
assert() |
||||
} |
||||
|
||||
func Test_EmptyCachedIterator(t *testing.T) { |
||||
|
||||
c := newCachedIterator(iter.NoopIterator, 0) |
||||
|
||||
require.Equal(t, "", c.Labels()) |
||||
require.Equal(t, logproto.Entry{}, c.Entry()) |
||||
require.Equal(t, false, c.Next()) |
||||
require.Equal(t, "", c.Labels()) |
||||
require.Equal(t, logproto.Entry{}, c.Entry()) |
||||
|
||||
require.Equal(t, nil, c.Close()) |
||||
|
||||
require.Equal(t, "", c.Labels()) |
||||
require.Equal(t, logproto.Entry{}, c.Entry()) |
||||
require.Equal(t, false, c.Next()) |
||||
require.Equal(t, "", c.Labels()) |
||||
require.Equal(t, logproto.Entry{}, c.Entry()) |
||||
|
||||
} |
||||
|
||||
func Test_ErrorCachedIterator(t *testing.T) { |
||||
|
||||
c := newCachedIterator(&errorIter{}, 0) |
||||
|
||||
require.Equal(t, false, c.Next()) |
||||
require.Equal(t, "", c.Labels()) |
||||
require.Equal(t, logproto.Entry{}, c.Entry()) |
||||
require.Equal(t, errors.New("error"), c.Error()) |
||||
require.Equal(t, errors.New("close"), c.Close()) |
||||
} |
||||
|
||||
type errorIter struct{} |
||||
|
||||
func (errorIter) Next() bool { return false } |
||||
func (errorIter) Error() error { return errors.New("error") } |
||||
func (errorIter) Labels() string { return "" } |
||||
func (errorIter) Entry() logproto.Entry { return logproto.Entry{} } |
||||
func (errorIter) Close() error { return errors.New("close") } |
@ -0,0 +1,157 @@ |
||||
package storage |
||||
|
||||
import ( |
||||
"context" |
||||
"errors" |
||||
"time" |
||||
|
||||
"github.com/cortexproject/cortex/pkg/chunk" |
||||
|
||||
"github.com/grafana/loki/pkg/chunkenc" |
||||
"github.com/grafana/loki/pkg/iter" |
||||
"github.com/grafana/loki/pkg/logproto" |
||||
"github.com/grafana/loki/pkg/logql" |
||||
) |
||||
|
||||
// LazyChunk loads the chunk when it is accessed.
|
||||
type LazyChunk struct { |
||||
Chunk chunk.Chunk |
||||
IsValid bool |
||||
Fetcher *chunk.Fetcher |
||||
|
||||
// cache of overlapping block.
|
||||
// We use the offset of the block as key since it's unique per chunk.
|
||||
overlappingBlocks map[int]*cachedIterator |
||||
} |
||||
|
||||
// Iterator returns an entry iterator.
|
||||
// The iterator returned will cache overlapping block's entries with the next chunk if passed.
|
||||
// This way when we re-use them for ordering across batches we don't re-decompress the data again.
|
||||
func (c *LazyChunk) Iterator( |
||||
ctx context.Context, |
||||
from, through time.Time, |
||||
direction logproto.Direction, |
||||
filter logql.LineFilter, |
||||
nextChunk *LazyChunk, |
||||
) (iter.EntryIterator, error) { |
||||
|
||||
// If the chunk is not already loaded, then error out.
|
||||
if c.Chunk.Data == nil { |
||||
return nil, errors.New("chunk is not loaded") |
||||
} |
||||
|
||||
lokiChunk := c.Chunk.Data.(*chunkenc.Facade).LokiChunk() |
||||
blocks := lokiChunk.Blocks(from, through) |
||||
if len(blocks) == 0 { |
||||
return iter.NoopIterator, nil |
||||
} |
||||
its := make([]iter.EntryIterator, 0, len(blocks)) |
||||
|
||||
for _, b := range blocks { |
||||
// if we have already processed and cache block let's use it.
|
||||
if cache, ok := c.overlappingBlocks[b.Offset()]; ok { |
||||
clone := *cache |
||||
clone.reset() |
||||
its = append(its, &clone) |
||||
continue |
||||
} |
||||
// if the block is overlapping cache it with the next chunk boundaries.
|
||||
if nextChunk != nil && IsBlockOverlapping(b, nextChunk, direction) { |
||||
it := newCachedIterator(b.Iterator(ctx, filter), b.Entries()) |
||||
its = append(its, it) |
||||
if c.overlappingBlocks == nil { |
||||
c.overlappingBlocks = make(map[int]*cachedIterator) |
||||
} |
||||
c.overlappingBlocks[b.Offset()] = it |
||||
continue |
||||
} |
||||
if nextChunk != nil { |
||||
delete(c.overlappingBlocks, b.Offset()) |
||||
} |
||||
// non-overlapping block with the next chunk are not cached.
|
||||
its = append(its, b.Iterator(ctx, filter)) |
||||
} |
||||
|
||||
// build the final iterator bound to the requested time range.
|
||||
iterForward := iter.NewTimeRangedIterator( |
||||
iter.NewNonOverlappingIterator(its, ""), |
||||
from, |
||||
through, |
||||
) |
||||
|
||||
if direction == logproto.FORWARD { |
||||
return iterForward, nil |
||||
} |
||||
|
||||
return iter.NewEntryReversedIter(iterForward) |
||||
} |
||||
|
||||
func IsBlockOverlapping(b chunkenc.Block, with *LazyChunk, direction logproto.Direction) bool { |
||||
if direction == logproto.BACKWARD { |
||||
through := int64(with.Chunk.Through) * int64(time.Millisecond) |
||||
if b.MinTime() <= through { |
||||
return true |
||||
} |
||||
} else { |
||||
from := int64(with.Chunk.From) * int64(time.Millisecond) |
||||
if b.MaxTime() >= from { |
||||
return true |
||||
} |
||||
} |
||||
return false |
||||
} |
||||
|
||||
func (c *LazyChunk) IsOverlapping(with *LazyChunk, direction logproto.Direction) bool { |
||||
if direction == logproto.BACKWARD { |
||||
if c.Chunk.From.Before(with.Chunk.Through) || c.Chunk.From == with.Chunk.Through { |
||||
return true |
||||
} |
||||
} else { |
||||
if !c.Chunk.Through.Before(with.Chunk.From) { |
||||
return true |
||||
} |
||||
} |
||||
return false |
||||
} |
||||
|
||||
// lazyChunks is a slice of lazy chunks that can ordered by chunk boundaries
|
||||
// in ascending or descending depending on the direction
|
||||
type lazyChunks struct { |
||||
chunks []*LazyChunk |
||||
direction logproto.Direction |
||||
} |
||||
|
||||
func (l lazyChunks) Len() int { return len(l.chunks) } |
||||
func (l lazyChunks) Swap(i, j int) { l.chunks[i], l.chunks[j] = l.chunks[j], l.chunks[i] } |
||||
func (l lazyChunks) Peek() *LazyChunk { return l.chunks[0] } |
||||
func (l lazyChunks) Less(i, j int) bool { |
||||
if l.direction == logproto.FORWARD { |
||||
t1, t2 := l.chunks[i].Chunk.From, l.chunks[j].Chunk.From |
||||
if !t1.Equal(t2) { |
||||
return t1.Before(t2) |
||||
} |
||||
return l.chunks[i].Chunk.Fingerprint < l.chunks[j].Chunk.Fingerprint |
||||
} |
||||
t1, t2 := l.chunks[i].Chunk.Through, l.chunks[j].Chunk.Through |
||||
if !t1.Equal(t2) { |
||||
return t1.After(t2) |
||||
} |
||||
return l.chunks[i].Chunk.Fingerprint > l.chunks[j].Chunk.Fingerprint |
||||
} |
||||
|
||||
// pop returns the top `count` lazychunks, the original slice is splitted an copied
|
||||
// to avoid retaining chunks in the slice backing array.
|
||||
func (l *lazyChunks) pop(count int) []*LazyChunk { |
||||
if len(l.chunks) <= count { |
||||
old := l.chunks |
||||
l.chunks = nil |
||||
return old |
||||
} |
||||
// split slices into two new ones and copy parts to each so we don't keep old reference
|
||||
res := make([]*LazyChunk, count) |
||||
copy(res, l.chunks[0:count]) |
||||
new := make([]*LazyChunk, len(l.chunks)-count) |
||||
copy(new, l.chunks[count:len(l.chunks)]) |
||||
l.chunks = new |
||||
return res |
||||
} |
@ -0,0 +1,115 @@ |
||||
package storage |
||||
|
||||
import ( |
||||
"context" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/cortexproject/cortex/pkg/chunk" |
||||
"github.com/stretchr/testify/require" |
||||
|
||||
"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/util" |
||||
) |
||||
|
||||
func TestIsOverlapping(t *testing.T) { |
||||
tests := []struct { |
||||
name string |
||||
direction logproto.Direction |
||||
with *LazyChunk |
||||
b chunkenc.Block |
||||
want bool |
||||
}{ |
||||
{ |
||||
"equal forward", |
||||
logproto.FORWARD, |
||||
lazyChunkWithBounds(time.Unix(0, 0), time.Unix(0, int64(time.Millisecond*5))), |
||||
blockWithBounds(0, int64(time.Millisecond*5)), |
||||
true, |
||||
}, |
||||
{ |
||||
"equal backward", |
||||
logproto.BACKWARD, |
||||
lazyChunkWithBounds(time.Unix(0, 0), time.Unix(0, int64(time.Millisecond*5))), |
||||
blockWithBounds(0, int64(time.Millisecond*5)), |
||||
true, |
||||
}, |
||||
{ |
||||
"equal through backward", |
||||
logproto.BACKWARD, |
||||
lazyChunkWithBounds(time.Unix(0, int64(time.Millisecond*5)), time.Unix(0, int64(time.Millisecond*10))), |
||||
blockWithBounds(0, int64(time.Millisecond*10)), |
||||
true, |
||||
}, |
||||
{ |
||||
"< through backward", |
||||
logproto.BACKWARD, |
||||
lazyChunkWithBounds(time.Unix(0, int64(time.Millisecond*5)), time.Unix(0, int64(time.Millisecond*10))), |
||||
blockWithBounds(0, int64(time.Millisecond*5)), |
||||
true, |
||||
}, |
||||
{ |
||||
"from > forward", |
||||
logproto.FORWARD, |
||||
lazyChunkWithBounds(time.Unix(0, int64(time.Millisecond*4)), time.Unix(0, int64(time.Millisecond*10))), |
||||
blockWithBounds(int64(time.Millisecond*3), int64(time.Millisecond*5)), |
||||
true, |
||||
}, |
||||
{ |
||||
"from < forward", |
||||
logproto.FORWARD, |
||||
lazyChunkWithBounds(time.Unix(0, int64(time.Millisecond*5)), time.Unix(0, int64(time.Millisecond*10))), |
||||
blockWithBounds(int64(time.Millisecond*3), int64(time.Millisecond*4)), |
||||
false, |
||||
}, |
||||
{ |
||||
"from = forward", |
||||
logproto.FORWARD, |
||||
lazyChunkWithBounds(time.Unix(0, int64(time.Millisecond*5)), time.Unix(0, int64(time.Millisecond*10))), |
||||
blockWithBounds(int64(time.Millisecond*3), int64(time.Millisecond*5)), |
||||
true, |
||||
}, |
||||
} |
||||
for _, tt := range tests { |
||||
t.Run(tt.name, func(t *testing.T) { |
||||
// testing the block one
|
||||
require.Equal(t, tt.want, IsBlockOverlapping(tt.b, tt.with, tt.direction)) |
||||
// testing the chunk one
|
||||
l := lazyChunkWithBounds(time.Unix(0, tt.b.MinTime()), time.Unix(0, tt.b.MaxTime())) |
||||
require.Equal(t, tt.want, l.IsOverlapping(tt.with, tt.direction)) |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func lazyChunkWithBounds(from, through time.Time) *LazyChunk { |
||||
// In loki chunks are rounded when flushed fro nanoseconds to milliseconds.
|
||||
fromM, throughM := util.RoundToMilliseconds(from, through) |
||||
return &LazyChunk{ |
||||
Chunk: chunk.Chunk{ |
||||
From: fromM, |
||||
Through: throughM, |
||||
}, |
||||
} |
||||
} |
||||
|
||||
type fakeBlock struct { |
||||
mint, maxt int64 |
||||
} |
||||
|
||||
func (fakeBlock) Entries() int { return 0 } |
||||
func (fakeBlock) Offset() int { return 0 } |
||||
func (f fakeBlock) MinTime() int64 { return f.mint } |
||||
func (f fakeBlock) MaxTime() int64 { return f.maxt } |
||||
func (fakeBlock) Iterator(context.Context, logql.LineFilter) iter.EntryIterator { |
||||
return nil |
||||
} |
||||
|
||||
func blockWithBounds(mint, maxt int64) chunkenc.Block { |
||||
return &fakeBlock{ |
||||
maxt: maxt, |
||||
mint: mint, |
||||
} |
||||
} |
Loading…
Reference in new issue