mirror of https://github.com/grafana/loki
Also adds more tests for the storepull/846/head
parent
9930f70446
commit
4d48972939
@ -0,0 +1,385 @@ |
||||
package storage |
||||
|
||||
import ( |
||||
"context" |
||||
"sort" |
||||
"time" |
||||
|
||||
"github.com/cortexproject/cortex/pkg/chunk" |
||||
"github.com/cortexproject/cortex/pkg/util/spanlogger" |
||||
"github.com/go-kit/kit/log/level" |
||||
"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/prometheus/common/model" |
||||
"github.com/prometheus/prometheus/pkg/labels" |
||||
) |
||||
|
||||
// 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 []*chunkenc.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() *chunkenc.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) []*chunkenc.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([]*chunkenc.LazyChunk, count) |
||||
copy(res, l.chunks[0:count]) |
||||
new := make([]*chunkenc.LazyChunk, len(l.chunks)-count) |
||||
copy(new, l.chunks[count:len(l.chunks)]) |
||||
l.chunks = new |
||||
return res |
||||
} |
||||
|
||||
// batchChunkIterator is an EntryIterator that iterates through chunks by batch of `batchSize`.
|
||||
// Since chunks can overlap across batches for each iteration the iterator will keep all overlapping
|
||||
// chunks with the next chunk from the next batch and added it to the next iteration. In this case the boundaries of the batch
|
||||
// is reduced to non-overlapping chunks boundaries.
|
||||
type batchChunkIterator struct { |
||||
chunks lazyChunks |
||||
batchSize int |
||||
err error |
||||
curr iter.EntryIterator |
||||
lastOverlapping []*chunkenc.LazyChunk |
||||
|
||||
ctx context.Context |
||||
matchers []*labels.Matcher |
||||
filter logql.Filter |
||||
req *logproto.QueryRequest |
||||
} |
||||
|
||||
// newBatchChunkIterator creates a new batch iterator with the given batchSize.
|
||||
func newBatchChunkIterator(ctx context.Context, chunks []*chunkenc.LazyChunk, batchSize int, matchers []*labels.Matcher, filter logql.Filter, req *logproto.QueryRequest) *batchChunkIterator { |
||||
res := &batchChunkIterator{ |
||||
batchSize: batchSize, |
||||
matchers: matchers, |
||||
filter: filter, |
||||
req: req, |
||||
ctx: ctx, |
||||
chunks: lazyChunks{direction: req.Direction, chunks: chunks}, |
||||
} |
||||
sort.Sort(res.chunks) |
||||
return res |
||||
} |
||||
|
||||
func (it *batchChunkIterator) Next() bool { |
||||
var err error |
||||
// for loop to avoid recursion
|
||||
for { |
||||
if it.curr != nil && it.curr.Next() { |
||||
return true |
||||
} |
||||
if it.chunks.Len() == 0 { |
||||
return false |
||||
} |
||||
// close previous iterator
|
||||
if it.curr != nil { |
||||
it.err = it.curr.Close() |
||||
} |
||||
it.curr, err = it.nextBatch() |
||||
if err != nil { |
||||
it.err = err |
||||
return false |
||||
} |
||||
} |
||||
} |
||||
|
||||
func (it *batchChunkIterator) nextBatch() (iter.EntryIterator, error) { |
||||
// pop the next batch of chunks and append/preprend previous overlapping chunks
|
||||
// so we can merge/de-dupe overlapping entries.
|
||||
batch := make([]*chunkenc.LazyChunk, 0, it.batchSize+len(it.lastOverlapping)) |
||||
if it.req.Direction == logproto.FORWARD { |
||||
batch = append(batch, it.lastOverlapping...) |
||||
} |
||||
batch = append(batch, it.chunks.pop(it.batchSize)...) |
||||
if it.req.Direction == logproto.BACKWARD { |
||||
batch = append(batch, it.lastOverlapping...) |
||||
} |
||||
|
||||
from, through := it.req.Start, it.req.End |
||||
if it.chunks.Len() > 0 { |
||||
nextChunk := it.chunks.Peek() |
||||
// we max out our iterator boundaries to the next chunks in the queue
|
||||
// so that overlapping chunks together
|
||||
if it.req.Direction == logproto.BACKWARD { |
||||
from = time.Unix(0, nextChunk.Chunk.Through.UnixNano()) |
||||
} else { |
||||
through = time.Unix(0, nextChunk.Chunk.From.UnixNano()) |
||||
} |
||||
// we save all overlapping chunks as they are also needed in the next batch to properly order entries.
|
||||
it.lastOverlapping = []*chunkenc.LazyChunk{} |
||||
for _, c := range batch { |
||||
if it.req.Direction == logproto.BACKWARD { |
||||
if c.Chunk.From.Before(nextChunk.Chunk.Through) || c.Chunk.From == nextChunk.Chunk.Through { |
||||
it.lastOverlapping = append(it.lastOverlapping, c) |
||||
} |
||||
} else { |
||||
if !c.Chunk.Through.Before(nextChunk.Chunk.From) { |
||||
it.lastOverlapping = append(it.lastOverlapping, c) |
||||
} |
||||
} |
||||
} |
||||
} else { |
||||
if len(it.lastOverlapping) > 0 { |
||||
if it.req.Direction == logproto.BACKWARD { |
||||
through = time.Unix(0, it.lastOverlapping[0].Chunk.From.UnixNano()) |
||||
} else { |
||||
from = time.Unix(0, it.lastOverlapping[0].Chunk.Through.UnixNano()) |
||||
} |
||||
} |
||||
} |
||||
|
||||
// create the new chunks iterator from the current batch.
|
||||
return newChunksIterator(it.ctx, batch, it.matchers, it.filter, it.req.Direction, from, through) |
||||
} |
||||
|
||||
func (it *batchChunkIterator) Entry() logproto.Entry { |
||||
return it.curr.Entry() |
||||
} |
||||
|
||||
func (it *batchChunkIterator) Labels() string { |
||||
return it.curr.Labels() |
||||
} |
||||
|
||||
func (it *batchChunkIterator) Error() error { |
||||
if it.err != nil { |
||||
return it.err |
||||
} |
||||
if it.curr != nil { |
||||
return it.curr.Error() |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
func (it *batchChunkIterator) Close() error { |
||||
if it.curr != nil { |
||||
return it.curr.Close() |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
// newChunksIterator creates an iterator over a set of lazychunks.
|
||||
func newChunksIterator(ctx context.Context, chunks []*chunkenc.LazyChunk, matchers []*labels.Matcher, filter logql.Filter, direction logproto.Direction, from, through time.Time) (iter.EntryIterator, error) { |
||||
chksBySeries := partitionBySeriesChunks(chunks) |
||||
|
||||
// Make sure the initial chunks are loaded. This is not one chunk
|
||||
// per series, but rather a chunk per non-overlapping iterator.
|
||||
if err := loadFirstChunks(ctx, chksBySeries); err != nil { |
||||
return nil, err |
||||
} |
||||
|
||||
// Now that we have the first chunk for each series loaded,
|
||||
// we can proceed to filter the series that don't match.
|
||||
chksBySeries = filterSeriesByMatchers(chksBySeries, matchers) |
||||
|
||||
var allChunks []*chunkenc.LazyChunk |
||||
for _, series := range chksBySeries { |
||||
for _, chunks := range series { |
||||
allChunks = append(allChunks, chunks...) |
||||
} |
||||
} |
||||
|
||||
// load all chunks not already loaded
|
||||
if err := fetchLazyChunks(ctx, allChunks); err != nil { |
||||
return nil, err |
||||
} |
||||
|
||||
iters, err := buildIterators(ctx, chksBySeries, filter, direction, from, through) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
|
||||
return iter.NewHeapIterator(iters, direction), nil |
||||
} |
||||
|
||||
func buildIterators(ctx context.Context, chks map[model.Fingerprint][][]*chunkenc.LazyChunk, filter logql.Filter, direction logproto.Direction, from, through time.Time) ([]iter.EntryIterator, error) { |
||||
result := make([]iter.EntryIterator, 0, len(chks)) |
||||
for _, chunks := range chks { |
||||
iterator, err := buildHeapIterator(ctx, chunks, filter, direction, from, through) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
result = append(result, iterator) |
||||
} |
||||
|
||||
return result, nil |
||||
} |
||||
|
||||
func buildHeapIterator(ctx context.Context, chks [][]*chunkenc.LazyChunk, filter logql.Filter, direction logproto.Direction, from, through time.Time) (iter.EntryIterator, error) { |
||||
result := make([]iter.EntryIterator, 0, len(chks)) |
||||
if chks[0][0].Chunk.Metric.Has("__name__") { |
||||
labelsBuilder := labels.NewBuilder(chks[0][0].Chunk.Metric) |
||||
labelsBuilder.Del("__name__") |
||||
chks[0][0].Chunk.Metric = labelsBuilder.Labels() |
||||
} |
||||
labels := chks[0][0].Chunk.Metric.String() |
||||
|
||||
for i := range chks { |
||||
iterators := make([]iter.EntryIterator, 0, len(chks[i])) |
||||
for j := range chks[i] { |
||||
iterator, err := chks[i][j].Iterator(ctx, from, through, direction, filter) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
iterators = append(iterators, iterator) |
||||
} |
||||
if direction == logproto.BACKWARD { |
||||
for i, j := 0, len(iterators)-1; i < j; i, j = i+1, j-1 { |
||||
iterators[i], iterators[j] = iterators[j], iterators[i] |
||||
} |
||||
} |
||||
result = append(result, iter.NewNonOverlappingIterator(iterators, labels)) |
||||
} |
||||
|
||||
return iter.NewHeapIterator(result, direction), nil |
||||
} |
||||
|
||||
func filterSeriesByMatchers(chks map[model.Fingerprint][][]*chunkenc.LazyChunk, matchers []*labels.Matcher) map[model.Fingerprint][][]*chunkenc.LazyChunk { |
||||
outer: |
||||
for fp, chunks := range chks { |
||||
for _, matcher := range matchers { |
||||
if !matcher.Matches(chunks[0][0].Chunk.Metric.Get(matcher.Name)) { |
||||
delete(chks, fp) |
||||
continue outer |
||||
} |
||||
} |
||||
} |
||||
return chks |
||||
} |
||||
|
||||
func fetchLazyChunks(ctx context.Context, chunks []*chunkenc.LazyChunk) error { |
||||
log, ctx := spanlogger.New(ctx, "LokiStore.fetchLazyChunks") |
||||
defer log.Finish() |
||||
|
||||
var totalChunks int |
||||
chksByFetcher := map[*chunk.Fetcher][]*chunkenc.LazyChunk{} |
||||
for _, c := range chunks { |
||||
if c.Chunk.Data == nil { |
||||
chksByFetcher[c.Fetcher] = append(chksByFetcher[c.Fetcher], c) |
||||
totalChunks++ |
||||
} |
||||
} |
||||
if len(chksByFetcher) == 0 { |
||||
return nil |
||||
} |
||||
level.Debug(log).Log("msg", "loading lazy chunks", "chunks", totalChunks) |
||||
|
||||
errChan := make(chan error) |
||||
for fetcher, chunks := range chksByFetcher { |
||||
go func(fetcher *chunk.Fetcher, chunks []*chunkenc.LazyChunk) { |
||||
|
||||
keys := make([]string, 0, len(chunks)) |
||||
chks := make([]chunk.Chunk, 0, len(chunks)) |
||||
index := make(map[string]*chunkenc.LazyChunk, len(chunks)) |
||||
|
||||
sort.Slice(chunks, func(i, j int) bool { return chunks[i].Chunk.ExternalKey() < chunks[j].Chunk.ExternalKey() }) |
||||
for _, chk := range chunks { |
||||
key := chk.Chunk.ExternalKey() |
||||
keys = append(keys, key) |
||||
chks = append(chks, chk.Chunk) |
||||
index[key] = chk |
||||
} |
||||
chks, err := fetcher.FetchChunks(ctx, chks, keys) |
||||
if err != nil { |
||||
errChan <- err |
||||
return |
||||
} |
||||
// assign fetched chunk by key as FetchChunks doesn't guarantee the order.
|
||||
for _, chk := range chks { |
||||
index[chk.ExternalKey()].Chunk = chk |
||||
} |
||||
|
||||
errChan <- nil |
||||
}(fetcher, chunks) |
||||
} |
||||
|
||||
var lastErr error |
||||
for i := 0; i < len(chksByFetcher); i++ { |
||||
if err := <-errChan; err != nil { |
||||
lastErr = err |
||||
} |
||||
} |
||||
|
||||
return lastErr |
||||
} |
||||
|
||||
func loadFirstChunks(ctx context.Context, chks map[model.Fingerprint][][]*chunkenc.LazyChunk) error { |
||||
var toLoad []*chunkenc.LazyChunk |
||||
for _, lchks := range chks { |
||||
for _, lchk := range lchks { |
||||
if len(lchk) == 0 { |
||||
continue |
||||
} |
||||
toLoad = append(toLoad, lchk[0]) |
||||
} |
||||
} |
||||
return fetchLazyChunks(ctx, toLoad) |
||||
} |
||||
|
||||
func partitionBySeriesChunks(chunks []*chunkenc.LazyChunk) map[model.Fingerprint][][]*chunkenc.LazyChunk { |
||||
chunksByFp := map[model.Fingerprint][]*chunkenc.LazyChunk{} |
||||
for _, c := range chunks { |
||||
fp := c.Chunk.Fingerprint |
||||
chunksByFp[fp] = append(chunksByFp[fp], c) |
||||
} |
||||
result := make(map[model.Fingerprint][][]*chunkenc.LazyChunk, len(chunksByFp)) |
||||
|
||||
for fp, chks := range chunksByFp { |
||||
result[fp] = partitionOverlappingChunks(chks) |
||||
} |
||||
|
||||
return result |
||||
} |
||||
|
||||
// partitionOverlappingChunks splits the list of chunks into different non-overlapping lists.
|
||||
// todo this might reverse the order.
|
||||
func partitionOverlappingChunks(chunks []*chunkenc.LazyChunk) [][]*chunkenc.LazyChunk { |
||||
sort.Slice(chunks, func(i, j int) bool { |
||||
return chunks[i].Chunk.From < chunks[j].Chunk.From |
||||
}) |
||||
|
||||
css := [][]*chunkenc.LazyChunk{} |
||||
outer: |
||||
for _, c := range chunks { |
||||
for i, cs := range css { |
||||
// If the chunk doesn't overlap with the current list, then add it to it.
|
||||
if cs[len(cs)-1].Chunk.Through.Before(c.Chunk.From) { |
||||
css[i] = append(css[i], c) |
||||
continue outer |
||||
} |
||||
} |
||||
// If the chunk overlaps with every existing list, then create a new list.
|
||||
cs := make([]*chunkenc.LazyChunk, 0, len(chunks)/(len(css)+1)) |
||||
cs = append(cs, c) |
||||
css = append(css, cs) |
||||
} |
||||
|
||||
return css |
||||
} |
@ -0,0 +1,309 @@ |
||||
package storage |
||||
|
||||
import ( |
||||
"context" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/grafana/loki/pkg/chunkenc" |
||||
"github.com/grafana/loki/pkg/iter" |
||||
"github.com/grafana/loki/pkg/logproto" |
||||
) |
||||
|
||||
func Test_newBatchChunkIterator(t *testing.T) { |
||||
|
||||
tests := map[string]struct { |
||||
chunks []*chunkenc.LazyChunk |
||||
expected []*logproto.Stream |
||||
matchers string |
||||
start, end time.Time |
||||
direction logproto.Direction |
||||
}{ |
||||
"forward with overlap": { |
||||
[]*chunkenc.LazyChunk{ |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from, |
||||
Line: "1", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
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", |
||||
}, |
||||
}, |
||||
}, |
||||
}, |
||||
fooLabels, |
||||
from, from.Add(3 * time.Millisecond), |
||||
logproto.FORWARD, |
||||
}, |
||||
"backward with overlap": { |
||||
[]*chunkenc.LazyChunk{ |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from, |
||||
Line: "1", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
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.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
{ |
||||
Timestamp: from, |
||||
Line: "1", |
||||
}, |
||||
}, |
||||
}, |
||||
}, |
||||
fooLabels, |
||||
from, from.Add(3 * time.Millisecond), |
||||
logproto.BACKWARD, |
||||
}, |
||||
"forward without overlap": { |
||||
[]*chunkenc.LazyChunk{ |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from, |
||||
Line: "1", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
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", |
||||
}, |
||||
}, |
||||
}, |
||||
}, |
||||
fooLabels, |
||||
from, from.Add(3 * time.Millisecond), |
||||
logproto.FORWARD, |
||||
}, |
||||
"backward without overlap": { |
||||
[]*chunkenc.LazyChunk{ |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from, |
||||
Line: "1", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
}, |
||||
}), |
||||
newLazyChunk(logproto.Stream{ |
||||
Labels: fooLabels, |
||||
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", |
||||
}, |
||||
}, |
||||
}, |
||||
}, |
||||
fooLabels, |
||||
from, from.Add(3 * time.Millisecond), |
||||
logproto.BACKWARD, |
||||
}, |
||||
} |
||||
|
||||
for name, tt := range tests { |
||||
tt := tt |
||||
t.Run(name, func(t *testing.T) { |
||||
it := newBatchChunkIterator(context.Background(), tt.chunks, 2, newMatchers(tt.matchers), nil, newQuery("", tt.start, tt.end, tt.direction)) |
||||
streams, _, err := iter.ReadBatch(it, 1000) |
||||
_ = it.Close() |
||||
if err != nil { |
||||
t.Fatalf("error reading batch %s", err) |
||||
} |
||||
|
||||
assertStream(t, tt.expected, streams.Streams) |
||||
|
||||
}) |
||||
} |
||||
|
||||
} |
@ -0,0 +1,245 @@ |
||||
package storage |
||||
|
||||
import ( |
||||
"context" |
||||
"sort" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/cortexproject/cortex/pkg/chunk" |
||||
"github.com/cortexproject/cortex/pkg/chunk/cache" |
||||
"github.com/cortexproject/cortex/pkg/ingester/client" |
||||
"github.com/davecgh/go-spew/spew" |
||||
"github.com/grafana/loki/pkg/chunkenc" |
||||
"github.com/grafana/loki/pkg/logproto" |
||||
"github.com/grafana/loki/pkg/logql" |
||||
"github.com/grafana/loki/pkg/util" |
||||
"github.com/prometheus/common/model" |
||||
"github.com/prometheus/prometheus/pkg/labels" |
||||
"github.com/stretchr/testify/assert" |
||||
) |
||||
|
||||
var fooLabels = "{foo=\"bar\"}" |
||||
|
||||
var from = time.Unix(0, time.Millisecond.Nanoseconds()) |
||||
|
||||
func assertStream(t *testing.T, expected, actual []*logproto.Stream) { |
||||
if len(expected) != len(actual) { |
||||
t.Fatalf("error stream length are different expected %d actual %d\n%s", len(expected), len(actual), spew.Sdump(expected, actual)) |
||||
return |
||||
} |
||||
sort.Slice(expected, func(i int, j int) bool { return expected[i].Labels < expected[j].Labels }) |
||||
sort.Slice(actual, func(i int, j int) bool { return actual[i].Labels < actual[j].Labels }) |
||||
for i := range expected { |
||||
assert.Equal(t, expected[i].Labels, actual[i].Labels) |
||||
if len(expected[i].Entries) != len(actual[i].Entries) { |
||||
t.Fatalf("error entries length are different expected %d actual%d\n%s", len(expected[i].Entries), len(actual[i].Entries), spew.Sdump(expected[i].Entries, actual[i].Entries)) |
||||
|
||||
return |
||||
} |
||||
for j := range expected[i].Entries { |
||||
assert.Equal(t, expected[i].Entries[j].Timestamp.UnixNano(), actual[i].Entries[j].Timestamp.UnixNano()) |
||||
assert.Equal(t, expected[i].Entries[j].Line, actual[i].Entries[j].Line) |
||||
} |
||||
} |
||||
} |
||||
|
||||
func newLazyChunk(stream logproto.Stream) *chunkenc.LazyChunk { |
||||
return &chunkenc.LazyChunk{ |
||||
Fetcher: nil, |
||||
Chunk: newChunk(stream), |
||||
} |
||||
} |
||||
|
||||
func newChunk(stream logproto.Stream) chunk.Chunk { |
||||
lbs, err := util.ToClientLabels(stream.Labels) |
||||
if err != nil { |
||||
panic(err) |
||||
} |
||||
l := client.FromLabelAdaptersToLabels(lbs) |
||||
if !l.Has(labels.MetricName) { |
||||
builder := labels.NewBuilder(l) |
||||
builder.Set(labels.MetricName, "logs") |
||||
l = builder.Labels() |
||||
} |
||||
from, through := model.TimeFromUnixNano(stream.Entries[0].Timestamp.UnixNano()), model.TimeFromUnixNano(stream.Entries[0].Timestamp.UnixNano()) |
||||
chk := chunkenc.NewMemChunk(chunkenc.EncGZIP) |
||||
for _, e := range stream.Entries { |
||||
if e.Timestamp.UnixNano() < from.UnixNano() { |
||||
from = model.TimeFromUnixNano(e.Timestamp.UnixNano()) |
||||
} |
||||
if e.Timestamp.UnixNano() > through.UnixNano() { |
||||
through = model.TimeFromUnixNano(e.Timestamp.UnixNano()) |
||||
} |
||||
_ = chk.Append(&e) |
||||
} |
||||
chk.Close() |
||||
c := chunk.NewChunk("fake", client.Fingerprint(l), l, chunkenc.NewFacade(chk), from, through) |
||||
// force the checksum creation
|
||||
if err := c.Encode(); err != nil { |
||||
panic(err) |
||||
} |
||||
return c |
||||
} |
||||
|
||||
func newMatchers(matchers string) []*labels.Matcher { |
||||
ls, err := logql.ParseExpr(matchers) |
||||
if err != nil { |
||||
panic(err) |
||||
} |
||||
return ls.Matchers() |
||||
} |
||||
|
||||
func newQuery(query string, start, end time.Time, direction logproto.Direction) *logproto.QueryRequest { |
||||
return &logproto.QueryRequest{ |
||||
Query: query, |
||||
Start: start, |
||||
Limit: 1000, |
||||
End: end, |
||||
Direction: direction, |
||||
} |
||||
} |
||||
|
||||
type mockChunkStore struct { |
||||
chunks []chunk.Chunk |
||||
} |
||||
|
||||
func newMockChunkStore(streams []*logproto.Stream) *mockChunkStore { |
||||
chunks := make([]chunk.Chunk, 0, len(streams)) |
||||
for _, s := range streams { |
||||
chunks = append(chunks, newChunk(*s)) |
||||
} |
||||
return &mockChunkStore{chunks: chunks} |
||||
} |
||||
func (m *mockChunkStore) Put(ctx context.Context, chunks []chunk.Chunk) error { return nil } |
||||
func (m *mockChunkStore) PutOne(ctx context.Context, from, through model.Time, chunk chunk.Chunk) error { |
||||
return nil |
||||
} |
||||
func (m *mockChunkStore) LabelValuesForMetricName(ctx context.Context, from, through model.Time, metricName string, labelName string) ([]string, error) { |
||||
return nil, nil |
||||
} |
||||
func (m *mockChunkStore) LabelNamesForMetricName(ctx context.Context, from, through model.Time, metricName string) ([]string, error) { |
||||
return nil, nil |
||||
} |
||||
func (m *mockChunkStore) Stop() {} |
||||
func (m *mockChunkStore) Get(ctx context.Context, from, through model.Time, matchers ...*labels.Matcher) ([]chunk.Chunk, error) { |
||||
return nil, nil |
||||
} |
||||
|
||||
// PutChunks implements ObjectClient from Fetcher
|
||||
func (m *mockChunkStore) PutChunks(ctx context.Context, chunks []chunk.Chunk) error { return nil } |
||||
|
||||
// GetChunks implements ObjectClient from Fetcher
|
||||
func (m *mockChunkStore) GetChunks(ctx context.Context, chunks []chunk.Chunk) ([]chunk.Chunk, error) { |
||||
var res []chunk.Chunk |
||||
for _, c := range chunks { |
||||
for _, sc := range m.chunks { |
||||
// only returns chunks requested using the external key
|
||||
if c.ExternalKey() == sc.ExternalKey() { |
||||
res = append(res, sc) |
||||
} |
||||
} |
||||
} |
||||
return res, nil |
||||
} |
||||
|
||||
func (m *mockChunkStore) GetChunkRefs(ctx context.Context, from, through model.Time, matchers ...*labels.Matcher) ([][]chunk.Chunk, []*chunk.Fetcher, error) { |
||||
refs := make([]chunk.Chunk, 0, len(m.chunks)) |
||||
// transform real chunks into ref chunks.
|
||||
for _, c := range m.chunks { |
||||
r, err := chunk.ParseExternalKey("fake", c.ExternalKey()) |
||||
if err != nil { |
||||
panic(err) |
||||
} |
||||
refs = append(refs, r) |
||||
} |
||||
f, err := chunk.NewChunkFetcher(cache.Config{}, false, m) |
||||
if err != nil { |
||||
panic(err) |
||||
} |
||||
return [][]chunk.Chunk{refs}, []*chunk.Fetcher{f}, nil |
||||
} |
||||
|
||||
var streamsFixture = []*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", |
||||
}, |
||||
}, |
||||
}, |
||||
{ |
||||
Labels: "{foo=\"bar\"}", |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(3 * time.Millisecond), |
||||
Line: "4", |
||||
}, |
||||
|
||||
{ |
||||
Timestamp: from.Add(4 * time.Millisecond), |
||||
Line: "5", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(5 * time.Millisecond), |
||||
Line: "6", |
||||
}, |
||||
}, |
||||
}, |
||||
{ |
||||
Labels: "{foo=\"bazz\"}", |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from, |
||||
Line: "1", |
||||
}, |
||||
|
||||
{ |
||||
Timestamp: from.Add(time.Millisecond), |
||||
Line: "2", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
}, |
||||
}, |
||||
{ |
||||
Labels: "{foo=\"bazz\"}", |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: from.Add(2 * time.Millisecond), |
||||
Line: "3", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(3 * time.Millisecond), |
||||
Line: "4", |
||||
}, |
||||
|
||||
{ |
||||
Timestamp: from.Add(4 * time.Millisecond), |
||||
Line: "5", |
||||
}, |
||||
{ |
||||
Timestamp: from.Add(5 * time.Millisecond), |
||||
Line: "6", |
||||
}, |
||||
}, |
||||
}, |
||||
} |
||||
var storeFixture = newMockChunkStore(streamsFixture) |
Loading…
Reference in new issue